View Javadoc

1   /**
2    * Licensed to the Apache Software Foundation (ASF) under one
3    * or more contributor license agreements.  See the NOTICE file
4    * distributed with this work for additional information
5    * regarding copyright ownership.  The ASF licenses this file
6    * to you under the Apache License, Version 2.0 (the
7    * "License"); you may not use this file except in compliance
8    * with the License.  You may obtain a copy of the License at
9    *
10   *     http://www.apache.org/licenses/LICENSE-2.0
11   *
12   * Unless required by applicable law or agreed to in writing, software
13   * distributed under the License is distributed on an "AS IS" BASIS,
14   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15   * See the License for the specific language governing permissions and
16   * limitations under the License.
17   */
18  package org.apache.hadoop.hbase.client;
19  
20  import static org.junit.Assert.assertTrue;
21  import static org.junit.Assert.fail;
22  
23  import java.io.IOException;
24  
25  import com.google.protobuf.ByteString;
26  import org.apache.commons.logging.Log;
27  import org.apache.commons.logging.LogFactory;
28  import org.apache.hadoop.conf.Configuration;
29  import org.apache.hadoop.hbase.TableName;
30  import org.apache.hadoop.hbase.HBaseConfiguration;
31  import org.apache.hadoop.hbase.HConstants;
32  import org.apache.hadoop.hbase.SmallTests;
33  import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
34  import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
35  import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
36  import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.IsSnapshotDoneRequest;
37  import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.IsSnapshotDoneResponse;
38  import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.TakeSnapshotRequest;
39  import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.TakeSnapshotResponse;
40  import org.junit.Test;
41  import org.junit.experimental.categories.Category;
42  import org.mockito.Mockito;
43  
44  import com.google.protobuf.RpcController;
45  
46  /**
47   * Test snapshot logic from the client
48   */
49  @Category(SmallTests.class)
50  public class TestSnapshotFromAdmin {
51  
52    private static final Log LOG = LogFactory.getLog(TestSnapshotFromAdmin.class);
53  
54    /**
55     * Test that the logic for doing 'correct' back-off based on exponential increase and the max-time
56     * passed from the server ensures the correct overall waiting for the snapshot to finish.
57     * @throws Exception
58     */
59    @Test(timeout = 60000)
60    public void testBackoffLogic() throws Exception {
61      final int pauseTime = 100;
62      final int maxWaitTime =
63        HConstants.RETRY_BACKOFF[HConstants.RETRY_BACKOFF.length - 1] * pauseTime;
64      final int numRetries = HConstants.RETRY_BACKOFF.length;
65      // calculate the wait time, if we just do straight backoff (ignoring the expected time from
66      // master)
67      long ignoreExpectedTime = 0;
68      for (int i = 0; i < HConstants.RETRY_BACKOFF.length; i++) {
69        ignoreExpectedTime += HConstants.RETRY_BACKOFF[i] * pauseTime;
70      }
71      // the correct wait time, capping at the maxTime/tries + fudge room
72      final long time = pauseTime * 3 + ((maxWaitTime / numRetries) * 3) + 300;
73      assertTrue("Capped snapshot wait time isn't less that the uncapped backoff time "
74          + "- further testing won't prove anything.", time < ignoreExpectedTime);
75  
76      // setup the mocks
77      HConnectionManager.HConnectionImplementation mockConnection = Mockito
78          .mock(HConnectionManager.HConnectionImplementation.class);
79      Configuration conf = HBaseConfiguration.create();
80      // setup the conf to match the expected properties
81      conf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, numRetries);
82      conf.setLong("hbase.client.pause", pauseTime);
83      // mock the master admin to our mock
84      MasterAdminKeepAliveConnection mockMaster = Mockito.mock(MasterAdminKeepAliveConnection.class);
85      Mockito.when(mockConnection.getConfiguration()).thenReturn(conf);
86      Mockito.when(mockConnection.getKeepAliveMasterAdminService()).thenReturn(mockMaster);
87      // set the max wait time for the snapshot to complete
88      TakeSnapshotResponse response = TakeSnapshotResponse.newBuilder()
89          .setExpectedTimeout(maxWaitTime)
90          .build();
91      Mockito
92          .when(
93            mockMaster.snapshot((RpcController) Mockito.isNull(),
94              Mockito.any(TakeSnapshotRequest.class))).thenReturn(response);
95      // setup the response
96      IsSnapshotDoneResponse.Builder builder = IsSnapshotDoneResponse.newBuilder();
97      builder.setDone(false);
98      // first five times, we return false, last we get success
99      Mockito.when(
100       mockMaster.isSnapshotDone((RpcController) Mockito.isNull(),
101         Mockito.any(IsSnapshotDoneRequest.class))).thenReturn(builder.build(), builder.build(),
102       builder.build(), builder.build(), builder.build(), builder.setDone(true).build());
103 
104     // setup the admin and run the test
105     HBaseAdmin admin = new HBaseAdmin(mockConnection);
106     String snapshot = "snapshot";
107     TableName table = TableName.valueOf("table");
108     // get start time
109     long start = System.currentTimeMillis();
110     admin.snapshot(snapshot, table);
111     long finish = System.currentTimeMillis();
112     long elapsed = (finish - start);
113     assertTrue("Elapsed time:" + elapsed + " is more than expected max:" + time, elapsed <= time);
114     admin.close();
115   }
116 
117   /**
118    * Make sure that we validate the snapshot name and the table name before we pass anything across
119    * the wire
120    * @throws Exception on failure
121    */
122   @Test
123   public void testValidateSnapshotName() throws Exception {
124     HConnectionManager.HConnectionImplementation mockConnection = Mockito
125         .mock(HConnectionManager.HConnectionImplementation.class);
126     Configuration conf = HBaseConfiguration.create();
127     Mockito.when(mockConnection.getConfiguration()).thenReturn(conf);
128     HBaseAdmin admin = new HBaseAdmin(mockConnection);
129     SnapshotDescription.Builder builder = SnapshotDescription.newBuilder();
130     // check that invalid snapshot names fail
131     failSnapshotStart(admin, builder.setName(HConstants.SNAPSHOT_DIR_NAME).build());
132     failSnapshotStart(admin, builder.setName("-snapshot").build());
133     failSnapshotStart(admin, builder.setName("snapshot fails").build());
134     failSnapshotStart(admin, builder.setName("snap$hot").build());
135     failSnapshotStart(admin, builder.setName("snap:hot").build());
136     // check the table name also get verified
137     failSnapshotStart(admin, builder.setName("snapshot").setTable(".table").build());
138     failSnapshotStart(admin, builder.setName("snapshot").setTable("-table").build());
139     failSnapshotStart(admin, builder.setName("snapshot").setTable("table fails").build());
140     failSnapshotStart(admin, builder.setName("snapshot").setTable("tab%le").build());
141 
142     // mock the master connection
143     MasterAdminKeepAliveConnection master = Mockito.mock(MasterAdminKeepAliveConnection.class);
144     Mockito.when(mockConnection.getKeepAliveMasterAdminService()).thenReturn(master);
145     TakeSnapshotResponse response = TakeSnapshotResponse.newBuilder().setExpectedTimeout(0).build();
146     Mockito.when(
147       master.snapshot((RpcController) Mockito.isNull(), Mockito.any(TakeSnapshotRequest.class)))
148         .thenReturn(response);
149     IsSnapshotDoneResponse doneResponse = IsSnapshotDoneResponse.newBuilder().setDone(true).build();
150     Mockito.when(
151       master.isSnapshotDone((RpcController) Mockito.isNull(),
152           Mockito.any(IsSnapshotDoneRequest.class))).thenReturn(doneResponse);
153 
154       // make sure that we can use valid names
155     admin.snapshot(builder.setName("snapshot").setTable("table").build());
156   }
157 
158   private void failSnapshotStart(HBaseAdmin admin, SnapshotDescription snapshot) throws IOException {
159     try {
160       admin.snapshot(snapshot);
161       fail("Snapshot should not have succeed with name:" + snapshot.getName());
162     } catch (IllegalArgumentException e) {
163       LOG.debug("Correctly failed to start snapshot:" + e.getMessage());
164     }
165   }
166 }