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