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 */ 018package org.apache.hadoop.hbase.master.procedure; 019 020import static org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.SnapshotState.SNAPSHOT_SNAPSHOT_ONLINE_REGIONS; 021import static org.junit.jupiter.api.Assertions.assertTrue; 022 023import java.io.IOException; 024import java.util.Optional; 025import java.util.concurrent.TimeUnit; 026import org.apache.hadoop.conf.Configuration; 027import org.apache.hadoop.hbase.HBaseTestingUtil; 028import org.apache.hadoop.hbase.TableName; 029import org.apache.hadoop.hbase.client.Admin; 030import org.apache.hadoop.hbase.client.SnapshotDescription; 031import org.apache.hadoop.hbase.client.SnapshotType; 032import org.apache.hadoop.hbase.client.Table; 033import org.apache.hadoop.hbase.master.HMaster; 034import org.apache.hadoop.hbase.procedure2.Procedure; 035import org.apache.hadoop.hbase.procedure2.ProcedureSuspendedException; 036import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility; 037import org.apache.hadoop.hbase.procedure2.ProcedureYieldException; 038import org.apache.hadoop.hbase.procedure2.RemoteProcedureDispatcher; 039import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils; 040import org.apache.hadoop.hbase.testclassification.MasterTests; 041import org.apache.hadoop.hbase.util.Bytes; 042import org.apache.hadoop.hbase.util.RegionSplitter; 043import org.junit.jupiter.api.AfterEach; 044import org.junit.jupiter.api.BeforeEach; 045import org.junit.jupiter.api.Tag; 046import org.mockito.Mockito; 047import org.mockito.internal.stubbing.answers.AnswersWithDelay; 048import org.mockito.invocation.InvocationOnMock; 049import org.mockito.stubbing.Answer; 050import org.slf4j.Logger; 051import org.slf4j.LoggerFactory; 052 053import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; 054import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos; 055import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.SnapshotState; 056import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos; 057 058@Tag(MasterTests.TAG) 059public class TestSnapshotProcedure { 060 protected static final Logger LOG = LoggerFactory.getLogger(TestSnapshotProcedure.class); 061 062 protected static HBaseTestingUtil TEST_UTIL; 063 protected HMaster master; 064 protected TableName TABLE_NAME; 065 protected byte[] CF; 066 protected String SNAPSHOT_NAME; 067 protected SnapshotDescription snapshot; 068 protected SnapshotProtos.SnapshotDescription snapshotProto; 069 protected Admin admin; 070 071 public static final class DelaySnapshotProcedure extends SnapshotProcedure { 072 public DelaySnapshotProcedure() { 073 } 074 075 public DelaySnapshotProcedure(final MasterProcedureEnv env, 076 final SnapshotProtos.SnapshotDescription snapshot) { 077 super(env, snapshot); 078 } 079 080 @Override 081 protected Flow executeFromState(MasterProcedureEnv env, 082 MasterProcedureProtos.SnapshotState state) 083 throws ProcedureSuspendedException, ProcedureYieldException, InterruptedException { 084 Flow flow = super.executeFromState(env, state); 085 if (state == SNAPSHOT_SNAPSHOT_ONLINE_REGIONS) { 086 TimeUnit.SECONDS.sleep(20); 087 } 088 return flow; 089 } 090 } 091 092 @BeforeEach 093 public void setup() throws Exception { 094 TEST_UTIL = new HBaseTestingUtil(); 095 Configuration config = TEST_UTIL.getConfiguration(); 096 // using SnapshotVerifyProcedure to verify snapshot 097 config.setInt("hbase.snapshot.remote.verify.threshold", 1); 098 // disable info server. Info server is useful when we run unit tests locally, but it will 099 // fails integration testing of jenkins. 100 // config.setInt(HConstants.MASTER_INFO_PORT, 8080); 101 102 // delay dispatch so that we can do something, for example kill a target server 103 config.setInt(RemoteProcedureDispatcher.DISPATCH_DELAY_CONF_KEY, 10000); 104 config.setInt(RemoteProcedureDispatcher.DISPATCH_MAX_QUEUE_SIZE_CONF_KEY, 128); 105 TEST_UTIL.startMiniCluster(3); 106 master = TEST_UTIL.getHBaseCluster().getMaster(); 107 admin = TEST_UTIL.getAdmin(); 108 TABLE_NAME = TableName.valueOf(Bytes.toBytes("SPTestTable")); 109 CF = Bytes.toBytes("cf"); 110 SNAPSHOT_NAME = "SnapshotProcedureTest"; 111 snapshot = new SnapshotDescription(SNAPSHOT_NAME, TABLE_NAME, SnapshotType.FLUSH); 112 snapshotProto = ProtobufUtil.createHBaseProtosSnapshotDesc(snapshot); 113 snapshotProto = SnapshotDescriptionUtils.validate(snapshotProto, master.getConfiguration()); 114 final byte[][] splitKeys = new RegionSplitter.HexStringSplit().split(10); 115 Table table = TEST_UTIL.createTable(TABLE_NAME, CF, splitKeys); 116 TEST_UTIL.loadTable(table, CF, false); 117 } 118 119 public <T extends Procedure<MasterProcedureEnv>> T 120 waitProcedureRunnableAndGetFirst(Class<T> clazz, long timeout) throws IOException { 121 TEST_UTIL.waitFor(timeout, () -> master.getProcedures().stream().anyMatch(clazz::isInstance)); 122 Optional<T> procOpt = master.getMasterProcedureExecutor().getProcedures().stream() 123 .filter(clazz::isInstance).map(clazz::cast).findFirst(); 124 assertTrue(procOpt.isPresent()); 125 return procOpt.get(); 126 } 127 128 protected SnapshotProcedure getDelayedOnSpecificStateSnapshotProcedure(SnapshotProcedure sp, 129 MasterProcedureEnv env, SnapshotState state) 130 throws ProcedureSuspendedException, ProcedureYieldException, InterruptedException { 131 SnapshotProcedure spySp = Mockito.spy(sp); 132 Mockito.doAnswer(new AnswersWithDelay(60000, new Answer<Object>() { 133 @Override 134 public Object answer(InvocationOnMock invocation) throws Throwable { 135 return invocation.callRealMethod(); 136 } 137 })).when(spySp).executeFromState(env, state); 138 return spySp; 139 } 140 141 @AfterEach 142 public void teardown() throws Exception { 143 if (this.master != null) { 144 ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(master.getMasterProcedureExecutor(), 145 false); 146 } 147 TEST_UTIL.shutdownMiniCluster(); 148 } 149}