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.jupiter.api.Assertions.assertNotEquals;
021import static org.junit.jupiter.api.Assertions.fail;
022
023import java.io.IOException;
024import org.apache.hadoop.conf.Configuration;
025import org.apache.hadoop.hbase.HBaseTestingUtil;
026import org.apache.hadoop.hbase.TableName;
027import org.apache.hadoop.hbase.client.SnapshotDescription;
028import org.apache.hadoop.hbase.client.SnapshotType;
029import org.apache.hadoop.hbase.client.Table;
030import org.apache.hadoop.hbase.master.HMaster;
031import org.apache.hadoop.hbase.master.snapshot.TakeSnapshotHandler;
032import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility;
033import org.apache.hadoop.hbase.snapshot.SnapshotTestingUtils;
034import org.apache.hadoop.hbase.testclassification.MasterTests;
035import org.apache.hadoop.hbase.testclassification.MediumTests;
036import org.apache.hadoop.hbase.util.Bytes;
037import org.apache.hadoop.hbase.util.RegionSplitter;
038import org.junit.jupiter.api.AfterEach;
039import org.junit.jupiter.api.BeforeEach;
040import org.junit.jupiter.api.Tag;
041import org.junit.jupiter.api.Test;
042import org.slf4j.Logger;
043import org.slf4j.LoggerFactory;
044
045import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
046import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos;
047import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos;
048
049/**
050 * Snapshot creation with master lock timeout test.
051 */
052@Tag(MasterTests.TAG)
053@Tag(MediumTests.TAG)
054public class TestSnapshotProcedureWithLockTimeout {
055
056  private static final Logger LOG =
057    LoggerFactory.getLogger(TestSnapshotProcedureWithLockTimeout.class);
058
059  private static HBaseTestingUtil TEST_UTIL;
060  private HMaster master;
061  private TableName TABLE_NAME;
062  private byte[] CF;
063  private String SNAPSHOT_NAME;
064
065  @BeforeEach
066  public void setup() throws Exception {
067    TEST_UTIL = new HBaseTestingUtil();
068    Configuration config = TEST_UTIL.getConfiguration();
069    config.setInt("hbase.snapshot.remote.verify.threshold", 1);
070    config.setLong(TakeSnapshotHandler.HBASE_SNAPSHOT_MASTER_LOCK_ACQUIRE_TIMEOUT, 1L);
071    TEST_UTIL.startMiniCluster(3);
072    master = TEST_UTIL.getHBaseCluster().getMaster();
073    TABLE_NAME = TableName.valueOf(Bytes.toBytes("TestSnapshotProcedureWithLockTimeout"));
074    CF = Bytes.toBytes("cf");
075    SNAPSHOT_NAME = "SnapshotProcLockTimeout";
076    final byte[][] splitKeys = new RegionSplitter.HexStringSplit().split(10);
077    Table table = TEST_UTIL.createTable(TABLE_NAME, CF, splitKeys);
078    TEST_UTIL.loadTable(table, CF, false);
079  }
080
081  @AfterEach
082  public void teardown() throws Exception {
083    if (this.master != null) {
084      ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(master.getMasterProcedureExecutor(),
085        false);
086    }
087    TEST_UTIL.shutdownMiniCluster();
088  }
089
090  @Test
091  public void testTakeZkCoordinatedSnapshot() {
092    for (int i = 0; i < 10; i++) {
093      try {
094        // Verify that snapshot creation is not possible because lock could not be
095        // acquired on time. This can be flaky behavior because even though we provide 1ms
096        // as lock timeout, it could still be fast enough and eventually lead to successful
097        // snapshot creation. If that happens, retry again.
098        testTakeZkCoordinatedSnapshot(i);
099        break;
100      } catch (Exception e) {
101        LOG.error("Error because of faster lock acquisition. retrying....", e);
102      }
103      assertNotEquals(9, i, "Retries exhausted");
104    }
105  }
106
107  private void testTakeZkCoordinatedSnapshot(int i) throws Exception {
108    SnapshotDescription snapshotOnSameTable =
109      new SnapshotDescription(SNAPSHOT_NAME + i, TABLE_NAME, SnapshotType.SKIPFLUSH);
110    SnapshotProtos.SnapshotDescription snapshotOnSameTableProto =
111      ProtobufUtil.createHBaseProtosSnapshotDesc(snapshotOnSameTable);
112    Thread second = new Thread("zk-snapshot") {
113      @Override
114      public void run() {
115        try {
116          master.getSnapshotManager().takeSnapshot(snapshotOnSameTableProto);
117        } catch (IOException e) {
118          LOG.error("zk snapshot failed", e);
119          fail("zk snapshot failed");
120        }
121      }
122    };
123    second.start();
124
125    Thread.sleep(5000);
126    boolean snapshotCreated = false;
127    try {
128      SnapshotTestingUtils.confirmSnapshotValid(TEST_UTIL, snapshotOnSameTableProto, TABLE_NAME,
129        CF);
130      snapshotCreated = true;
131    } catch (AssertionError e) {
132      LOG.error("Assertion error..", e);
133      if (
134        e.getMessage() != null && e.getMessage().contains("target snapshot directory")
135          && e.getMessage().contains("doesn't exist.")
136      ) {
137        LOG.debug("Expected behaviour - snapshot could not be created");
138      } else {
139        throw new IOException(e);
140      }
141    }
142
143    if (snapshotCreated) {
144      throw new IOException("Snapshot created successfully");
145    }
146
147    // ensure all scheduled procedures are successfully completed
148    TEST_UTIL.waitFor(4000, 400,
149      () -> master.getMasterProcedureExecutor().getProcedures().stream()
150        .filter(masterProcedureEnvProcedure -> masterProcedureEnvProcedure.getState()
151            == ProcedureProtos.ProcedureState.SUCCESS)
152        .count() == master.getMasterProcedureExecutor().getProcedures().size());
153  }
154}