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.junit.Assert.assertEquals;
021import static org.junit.Assert.assertFalse;
022import static org.junit.Assert.fail;
023
024import java.io.IOException;
025import java.util.Arrays;
026import java.util.Comparator;
027import java.util.List;
028import java.util.stream.Collectors;
029import org.apache.hadoop.hbase.HBaseClassTestRule;
030import org.apache.hadoop.hbase.client.SnapshotDescription;
031import org.apache.hadoop.hbase.client.SnapshotType;
032import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
033import org.apache.hadoop.hbase.procedure2.Procedure;
034import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
035import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility;
036import org.apache.hadoop.hbase.snapshot.SnapshotTestingUtils;
037import org.apache.hadoop.hbase.testclassification.MasterTests;
038import org.apache.hadoop.hbase.testclassification.MediumTests;
039import org.junit.ClassRule;
040import org.junit.Test;
041import org.junit.experimental.categories.Category;
042
043import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
044import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos;
045import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos;
046
047@Category({ MasterTests.class, MediumTests.class })
048public class TestSnapshotProcedureConcurrently extends TestSnapshotProcedure {
049
050  @ClassRule
051  public static final HBaseClassTestRule CLASS_RULE =
052    HBaseClassTestRule.forClass(TestSnapshotProcedureConcurrently.class);
053
054  @Test
055  public void testRunningTwoSnapshotProcedureOnSameTable() throws Exception {
056    String newSnapshotName = SNAPSHOT_NAME + "_2";
057    SnapshotProtos.SnapshotDescription snapshotProto2 =
058      SnapshotProtos.SnapshotDescription.newBuilder(snapshotProto).setName(newSnapshotName).build();
059
060    ProcedureExecutor<MasterProcedureEnv> procExec = master.getMasterProcedureExecutor();
061    MasterProcedureEnv env = procExec.getEnvironment();
062
063    SnapshotProcedure sp1 = new SnapshotProcedure(env, snapshotProto);
064    SnapshotProcedure sp2 = new SnapshotProcedure(env, snapshotProto2);
065    SnapshotProcedure spySp1 =
066      getDelayedOnSpecificStateSnapshotProcedure(sp1, procExec.getEnvironment(),
067        MasterProcedureProtos.SnapshotState.SNAPSHOT_SNAPSHOT_ONLINE_REGIONS);
068    SnapshotProcedure spySp2 =
069      getDelayedOnSpecificStateSnapshotProcedure(sp2, procExec.getEnvironment(),
070        MasterProcedureProtos.SnapshotState.SNAPSHOT_SNAPSHOT_ONLINE_REGIONS);
071
072    long procId1 = procExec.submitProcedure(spySp1);
073    long procId2 = procExec.submitProcedure(spySp2);
074    TEST_UTIL.waitFor(2000,
075      () -> env.getMasterServices().getProcedures().stream().map(Procedure::getProcId)
076        .collect(Collectors.toList()).containsAll(Arrays.asList(procId1, procId2)));
077
078    assertFalse(procExec.isFinished(procId1));
079    assertFalse(procExec.isFinished(procId2));
080
081    ProcedureTestingUtility.waitProcedure(master.getMasterProcedureExecutor(), procId1);
082    ProcedureTestingUtility.waitProcedure(master.getMasterProcedureExecutor(), procId2);
083
084    List<SnapshotProtos.SnapshotDescription> snapshots =
085      master.getSnapshotManager().getCompletedSnapshots();
086    assertEquals(2, snapshots.size());
087    snapshots.sort(Comparator.comparing(SnapshotProtos.SnapshotDescription::getName));
088    assertEquals(SNAPSHOT_NAME, snapshots.get(0).getName());
089    assertEquals(newSnapshotName, snapshots.get(1).getName());
090    SnapshotTestingUtils.confirmSnapshotValid(TEST_UTIL, snapshotProto, TABLE_NAME, CF);
091    SnapshotTestingUtils.confirmSnapshotValid(TEST_UTIL, snapshotProto2, TABLE_NAME, CF);
092  }
093
094  @Test
095  public void testTakeZkCoordinatedSnapshotAndProcedureCoordinatedSnapshotBoth() throws Exception {
096    String newSnapshotName = SNAPSHOT_NAME + "_2";
097    Thread first = new Thread("procedure-snapshot") {
098      @Override
099      public void run() {
100        try {
101          TEST_UTIL.getAdmin().snapshot(snapshot);
102        } catch (IOException e) {
103          LOG.error("procedure snapshot failed", e);
104          fail("procedure snapshot failed");
105        }
106      }
107    };
108    first.start();
109    Thread.sleep(1000);
110
111    SnapshotManager sm = master.getSnapshotManager();
112    TEST_UTIL.waitFor(2000, 50,
113      () -> !sm.isTakingSnapshot(TABLE_NAME) && sm.isTableTakingAnySnapshot(TABLE_NAME));
114
115    TEST_UTIL.getConfiguration().setBoolean("hbase.snapshot.zk.coordinated", true);
116    SnapshotDescription snapshotOnSameTable =
117      new SnapshotDescription(newSnapshotName, TABLE_NAME, SnapshotType.SKIPFLUSH);
118    SnapshotProtos.SnapshotDescription snapshotOnSameTableProto =
119      ProtobufUtil.createHBaseProtosSnapshotDesc(snapshotOnSameTable);
120    Thread second = new Thread("zk-snapshot") {
121      @Override
122      public void run() {
123        try {
124          master.getSnapshotManager().takeSnapshot(snapshotOnSameTableProto);
125        } catch (IOException e) {
126          LOG.error("zk snapshot failed", e);
127          fail("zk snapshot failed");
128        }
129      }
130    };
131    second.start();
132
133    TEST_UTIL.waitFor(2000, () -> sm.isTakingSnapshot(TABLE_NAME));
134    TEST_UTIL.waitFor(60000,
135      () -> sm.isSnapshotDone(snapshotOnSameTableProto) && !sm.isTakingAnySnapshot());
136    SnapshotTestingUtils.confirmSnapshotValid(TEST_UTIL, snapshotProto, TABLE_NAME, CF);
137    SnapshotTestingUtils.confirmSnapshotValid(TEST_UTIL, snapshotOnSameTableProto, TABLE_NAME, CF);
138  }
139}