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.EnableTableState.ENABLE_TABLE_MARK_REGIONS_ONLINE; 021import static org.junit.Assert.assertEquals; 022import static org.junit.Assert.assertFalse; 023import static org.junit.Assert.fail; 024 025import java.io.IOException; 026import java.util.Arrays; 027import java.util.Comparator; 028import java.util.List; 029import java.util.stream.Collectors; 030import org.apache.hadoop.hbase.HBaseClassTestRule; 031import org.apache.hadoop.hbase.TableName; 032import org.apache.hadoop.hbase.client.SnapshotDescription; 033import org.apache.hadoop.hbase.client.SnapshotType; 034import org.apache.hadoop.hbase.master.snapshot.SnapshotManager; 035import org.apache.hadoop.hbase.procedure2.Procedure; 036import org.apache.hadoop.hbase.procedure2.ProcedureExecutor; 037import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility; 038import org.apache.hadoop.hbase.snapshot.SnapshotTestingUtils; 039import org.apache.hadoop.hbase.testclassification.MasterTests; 040import org.apache.hadoop.hbase.testclassification.MediumTests; 041import org.junit.ClassRule; 042import org.junit.Test; 043import org.junit.experimental.categories.Category; 044 045import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; 046import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos; 047import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos; 048 049@Category({ MasterTests.class, MediumTests.class }) 050public class TestSnapshotProcedureConcurrently extends TestSnapshotProcedure { 051 052 @ClassRule 053 public static final HBaseClassTestRule CLASS_RULE = 054 HBaseClassTestRule.forClass(TestSnapshotProcedureConcurrently.class); 055 056 @Test 057 public void testRunningTwoSnapshotProcedureOnSameTable() throws Exception { 058 String newSnapshotName = SNAPSHOT_NAME + "_2"; 059 SnapshotProtos.SnapshotDescription snapshotProto2 = 060 SnapshotProtos.SnapshotDescription.newBuilder(snapshotProto).setName(newSnapshotName).build(); 061 062 ProcedureExecutor<MasterProcedureEnv> procExec = master.getMasterProcedureExecutor(); 063 MasterProcedureEnv env = procExec.getEnvironment(); 064 065 SnapshotProcedure sp1 = new SnapshotProcedure(env, snapshotProto); 066 SnapshotProcedure sp2 = new SnapshotProcedure(env, snapshotProto2); 067 SnapshotProcedure spySp1 = 068 getDelayedOnSpecificStateSnapshotProcedure(sp1, procExec.getEnvironment(), 069 MasterProcedureProtos.SnapshotState.SNAPSHOT_SNAPSHOT_ONLINE_REGIONS); 070 SnapshotProcedure spySp2 = 071 getDelayedOnSpecificStateSnapshotProcedure(sp2, procExec.getEnvironment(), 072 MasterProcedureProtos.SnapshotState.SNAPSHOT_SNAPSHOT_ONLINE_REGIONS); 073 074 long procId1 = procExec.submitProcedure(spySp1); 075 long procId2 = procExec.submitProcedure(spySp2); 076 TEST_UTIL.waitFor(2000, 077 () -> env.getMasterServices().getProcedures().stream().map(Procedure::getProcId) 078 .collect(Collectors.toList()).containsAll(Arrays.asList(procId1, procId2))); 079 080 assertFalse(procExec.isFinished(procId1)); 081 assertFalse(procExec.isFinished(procId2)); 082 083 ProcedureTestingUtility.waitProcedure(master.getMasterProcedureExecutor(), procId1); 084 ProcedureTestingUtility.waitProcedure(master.getMasterProcedureExecutor(), procId2); 085 086 List<SnapshotProtos.SnapshotDescription> snapshots = 087 master.getSnapshotManager().getCompletedSnapshots(); 088 assertEquals(2, snapshots.size()); 089 snapshots.sort(Comparator.comparing(SnapshotProtos.SnapshotDescription::getName)); 090 assertEquals(SNAPSHOT_NAME, snapshots.get(0).getName()); 091 assertEquals(newSnapshotName, snapshots.get(1).getName()); 092 SnapshotTestingUtils.confirmSnapshotValid(TEST_UTIL, snapshotProto, TABLE_NAME, CF); 093 SnapshotTestingUtils.confirmSnapshotValid(TEST_UTIL, snapshotProto2, TABLE_NAME, CF); 094 } 095 096 @Test 097 public void testTakeZkCoordinatedSnapshotAndProcedureCoordinatedSnapshotBoth() throws Exception { 098 String newSnapshotName = SNAPSHOT_NAME + "_2"; 099 Thread first = new Thread("procedure-snapshot") { 100 @Override 101 public void run() { 102 try { 103 TEST_UTIL.getAdmin().snapshot(snapshot); 104 } catch (IOException e) { 105 LOG.error("procedure snapshot failed", e); 106 fail("procedure snapshot failed"); 107 } 108 } 109 }; 110 first.start(); 111 Thread.sleep(1000); 112 113 SnapshotManager sm = master.getSnapshotManager(); 114 TEST_UTIL.waitFor(2000, 50, 115 () -> !sm.isTakingSnapshot(TABLE_NAME) && sm.isTableTakingAnySnapshot(TABLE_NAME)); 116 117 TEST_UTIL.getConfiguration().setBoolean("hbase.snapshot.zk.coordinated", true); 118 SnapshotDescription snapshotOnSameTable = 119 new SnapshotDescription(newSnapshotName, TABLE_NAME, SnapshotType.SKIPFLUSH); 120 SnapshotProtos.SnapshotDescription snapshotOnSameTableProto = 121 ProtobufUtil.createHBaseProtosSnapshotDesc(snapshotOnSameTable); 122 Thread second = new Thread("zk-snapshot") { 123 @Override 124 public void run() { 125 try { 126 master.getSnapshotManager().takeSnapshot(snapshotOnSameTableProto); 127 } catch (IOException e) { 128 LOG.error("zk snapshot failed", e); 129 fail("zk snapshot failed"); 130 } 131 } 132 }; 133 second.start(); 134 135 TEST_UTIL.waitFor(2000, () -> sm.isTakingSnapshot(TABLE_NAME)); 136 TEST_UTIL.waitFor(60000, 137 () -> sm.isSnapshotDone(snapshotOnSameTableProto) && !sm.isTakingAnySnapshot()); 138 SnapshotTestingUtils.confirmSnapshotValid(TEST_UTIL, snapshotProto, TABLE_NAME, CF); 139 SnapshotTestingUtils.confirmSnapshotValid(TEST_UTIL, snapshotOnSameTableProto, TABLE_NAME, CF); 140 } 141 142 @Test 143 public void testItFailsIfTableIsNotDisabledOrEnabled() throws Exception { 144 ProcedureExecutor<MasterProcedureEnv> executor = master.getMasterProcedureExecutor(); 145 MasterProcedureEnv env = executor.getEnvironment(); 146 TEST_UTIL.getAdmin().disableTable(TABLE_NAME); 147 148 TestEnableTableProcedure enableTable = new TestEnableTableProcedure( 149 master.getMasterProcedureExecutor().getEnvironment(), TABLE_NAME); 150 long enableProcId = executor.submitProcedure(enableTable); 151 TEST_UTIL.waitFor(60000, () -> { 152 Procedure<MasterProcedureEnv> proc = executor.getProcedure(enableProcId); 153 if (proc == null) { 154 return false; 155 } 156 return ((TestEnableTableProcedure) proc).getProcedureState() 157 == ENABLE_TABLE_MARK_REGIONS_ONLINE; 158 }); 159 160 // Using a delayed spy ensures we hit the problem state while the table enable procedure 161 // is waiting to run 162 SnapshotProcedure snapshotProc = new SnapshotProcedure(env, snapshotProto); 163 long snapshotProcId = executor.submitProcedure(snapshotProc); 164 TEST_UTIL.waitTableEnabled(TABLE_NAME); 165 // Wait for procedure to run and finish 166 TEST_UTIL.waitFor(60000, () -> executor.getProcedure(snapshotProcId) != null); 167 TEST_UTIL.waitFor(60000, () -> executor.getProcedure(snapshotProcId) == null); 168 169 SnapshotTestingUtils.confirmSnapshotValid(TEST_UTIL, snapshotProto, TABLE_NAME, CF); 170 } 171 172 // Needs to be publicly accessible for Procedure validation 173 public static class TestEnableTableProcedure extends EnableTableProcedure { 174 // Necessary for Procedure validation 175 public TestEnableTableProcedure() { 176 } 177 178 public TestEnableTableProcedure(MasterProcedureEnv env, TableName tableName) { 179 super(env, tableName); 180 } 181 182 public MasterProcedureProtos.EnableTableState getProcedureState() { 183 return getState(stateCount); 184 } 185 186 @Override 187 protected Flow executeFromState(MasterProcedureEnv env, 188 MasterProcedureProtos.EnableTableState state) throws InterruptedException { 189 if (state == ENABLE_TABLE_MARK_REGIONS_ONLINE) { 190 Thread.sleep(10000); 191 } 192 193 return super.executeFromState(env, state); 194 } 195 } 196}