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.client; 019 020import static org.junit.Assert.assertTrue; 021import static org.junit.Assert.fail; 022 023import java.io.IOException; 024import org.apache.hadoop.conf.Configuration; 025import org.apache.hadoop.hbase.HBaseClassTestRule; 026import org.apache.hadoop.hbase.HBaseConfiguration; 027import org.apache.hadoop.hbase.HConstants; 028import org.apache.hadoop.hbase.TableName; 029import org.apache.hadoop.hbase.ipc.HBaseRpcController; 030import org.apache.hadoop.hbase.ipc.RpcControllerFactory; 031import org.apache.hadoop.hbase.testclassification.ClientTests; 032import org.apache.hadoop.hbase.testclassification.SmallTests; 033import org.junit.ClassRule; 034import org.junit.Rule; 035import org.junit.Test; 036import org.junit.experimental.categories.Category; 037import org.junit.rules.TestName; 038import org.mockito.Mockito; 039import org.slf4j.Logger; 040import org.slf4j.LoggerFactory; 041 042import org.apache.hbase.thirdparty.com.google.protobuf.RpcController; 043 044import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSnapshotDoneResponse; 045import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SnapshotResponse; 046 047/** 048 * Test snapshot logic from the client 049 */ 050@Category({SmallTests.class, ClientTests.class}) 051public class TestSnapshotFromAdmin { 052 053 @ClassRule 054 public static final HBaseClassTestRule CLASS_RULE = 055 HBaseClassTestRule.forClass(TestSnapshotFromAdmin.class); 056 057 private static final Logger LOG = LoggerFactory.getLogger(TestSnapshotFromAdmin.class); 058 059 @Rule 060 public TestName name = new TestName(); 061 062 /** 063 * Test that the logic for doing 'correct' back-off based on exponential increase and the max-time 064 * passed from the server ensures the correct overall waiting for the snapshot to finish. 065 * @throws Exception 066 */ 067 @Test 068 public void testBackoffLogic() throws Exception { 069 final int pauseTime = 100; 070 final int maxWaitTime = 071 HConstants.RETRY_BACKOFF[HConstants.RETRY_BACKOFF.length - 1] * pauseTime; 072 final int numRetries = HConstants.RETRY_BACKOFF.length; 073 // calculate the wait time, if we just do straight backoff (ignoring the expected time from 074 // master) 075 long ignoreExpectedTime = 0; 076 for (int i = 0; i < HConstants.RETRY_BACKOFF.length; i++) { 077 ignoreExpectedTime += HConstants.RETRY_BACKOFF[i] * pauseTime; 078 } 079 // the correct wait time, capping at the maxTime/tries + fudge room 080 final long time = pauseTime * 3L + ((maxWaitTime / numRetries) * 3) + 300L; 081 assertTrue("Capped snapshot wait time isn't less that the uncapped backoff time " 082 + "- further testing won't prove anything.", time < ignoreExpectedTime); 083 084 // setup the mocks 085 ConnectionImplementation mockConnection = Mockito 086 .mock(ConnectionImplementation.class); 087 Configuration conf = HBaseConfiguration.create(); 088 // setup the conf to match the expected properties 089 conf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, numRetries); 090 conf.setLong("hbase.client.pause", pauseTime); 091 092 // mock the master admin to our mock 093 MasterKeepAliveConnection mockMaster = Mockito.mock(MasterKeepAliveConnection.class); 094 Mockito.when(mockConnection.getConfiguration()).thenReturn(conf); 095 Mockito.when(mockConnection.getMaster()).thenReturn(mockMaster); 096 // we need a real retrying caller 097 RpcRetryingCallerFactory callerFactory = new RpcRetryingCallerFactory(conf); 098 RpcControllerFactory controllerFactory = Mockito.mock(RpcControllerFactory.class); 099 Mockito.when(controllerFactory.newController()).thenReturn( 100 Mockito.mock(HBaseRpcController.class)); 101 Mockito.when(mockConnection.getRpcRetryingCallerFactory()).thenReturn(callerFactory); 102 Mockito.when(mockConnection.getRpcControllerFactory()).thenReturn(controllerFactory); 103 // set the max wait time for the snapshot to complete 104 SnapshotResponse response = SnapshotResponse.newBuilder() 105 .setExpectedTimeout(maxWaitTime) 106 .build(); 107 Mockito 108 .when( 109 mockMaster.snapshot((RpcController) Mockito.any(), 110 Mockito.any())).thenReturn(response); 111 // setup the response 112 IsSnapshotDoneResponse.Builder builder = IsSnapshotDoneResponse.newBuilder(); 113 builder.setDone(false); 114 // first five times, we return false, last we get success 115 Mockito.when( 116 mockMaster.isSnapshotDone((RpcController) Mockito.any(), 117 Mockito.any())).thenReturn(builder.build(), builder.build(), 118 builder.build(), builder.build(), builder.build(), builder.setDone(true).build()); 119 120 // setup the admin and run the test 121 Admin admin = new HBaseAdmin(mockConnection); 122 String snapshot = "snapshot"; 123 final TableName table = TableName.valueOf(name.getMethodName()); 124 // get start time 125 long start = System.currentTimeMillis(); 126 admin.snapshot(snapshot, table); 127 long finish = System.currentTimeMillis(); 128 long elapsed = (finish - start); 129 assertTrue("Elapsed time:" + elapsed + " is more than expected max:" + time, elapsed <= time); 130 admin.close(); 131 } 132 133 /** 134 * Make sure that we validate the snapshot name and the table name before we pass anything across 135 * the wire 136 * @throws Exception on failure 137 */ 138 @Test 139 public void testValidateSnapshotName() throws Exception { 140 ConnectionImplementation mockConnection = Mockito 141 .mock(ConnectionImplementation.class); 142 Configuration conf = HBaseConfiguration.create(); 143 Mockito.when(mockConnection.getConfiguration()).thenReturn(conf); 144 // we need a real retrying caller 145 RpcRetryingCallerFactory callerFactory = new RpcRetryingCallerFactory(conf); 146 RpcControllerFactory controllerFactory = Mockito.mock(RpcControllerFactory.class); 147 Mockito.when(controllerFactory.newController()).thenReturn( 148 Mockito.mock(HBaseRpcController.class)); 149 Mockito.when(mockConnection.getRpcRetryingCallerFactory()).thenReturn(callerFactory); 150 Mockito.when(mockConnection.getRpcControllerFactory()).thenReturn(controllerFactory); 151 Admin admin = new HBaseAdmin(mockConnection); 152 // check that invalid snapshot names fail 153 failSnapshotStart(admin, new SnapshotDescription(HConstants.SNAPSHOT_DIR_NAME)); 154 failSnapshotStart(admin, new SnapshotDescription("-snapshot")); 155 failSnapshotStart(admin, new SnapshotDescription("snapshot fails")); 156 failSnapshotStart(admin, new SnapshotDescription("snap$hot")); 157 failSnapshotStart(admin, new SnapshotDescription("snap:hot")); 158 // check the table name also get verified 159 failSnapshotDescriptorCreation("snapshot", ".table"); 160 failSnapshotDescriptorCreation("snapshot", "-table"); 161 failSnapshotDescriptorCreation("snapshot", "table fails"); 162 failSnapshotDescriptorCreation("snapshot", "tab%le"); 163 164 // mock the master connection 165 MasterKeepAliveConnection master = Mockito.mock(MasterKeepAliveConnection.class); 166 Mockito.when(mockConnection.getMaster()).thenReturn(master); 167 SnapshotResponse response = SnapshotResponse.newBuilder().setExpectedTimeout(0).build(); 168 Mockito.when( 169 master.snapshot((RpcController) Mockito.any(), Mockito.any())) 170 .thenReturn(response); 171 IsSnapshotDoneResponse doneResponse = IsSnapshotDoneResponse.newBuilder().setDone(true).build(); 172 Mockito.when( 173 master.isSnapshotDone((RpcController) Mockito.any(), 174 Mockito.any())).thenReturn(doneResponse); 175 176 // make sure that we can use valid names 177 admin.snapshot(new SnapshotDescription("snapshot", TableName.valueOf(name.getMethodName()))); 178 } 179 180 private void failSnapshotStart(Admin admin, SnapshotDescription snapshot) 181 throws IOException { 182 try { 183 admin.snapshot(snapshot); 184 fail("Snapshot should not have succeed with name:" + snapshot.getName()); 185 } catch (IllegalArgumentException e) { 186 LOG.debug("Correctly failed to start snapshot:" + e.getMessage()); 187 } 188 } 189 190 private void failSnapshotDescriptorCreation(final String snapshotName, final String tableName) { 191 try { 192 new SnapshotDescription(snapshotName, tableName); 193 fail("SnapshotDescription should not have succeed with name:" + snapshotName); 194 } catch (IllegalArgumentException e) { 195 LOG.debug("Correctly failed to create SnapshotDescription:" + e.getMessage()); 196 } 197 } 198}