1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
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 org.apache.commons.logging.Log;
26 import org.apache.commons.logging.LogFactory;
27 import org.apache.hadoop.conf.Configuration;
28 import org.apache.hadoop.hbase.HBaseConfiguration;
29 import org.apache.hadoop.hbase.HConstants;
30 import org.apache.hadoop.hbase.testclassification.SmallTests;
31 import org.apache.hadoop.hbase.TableName;
32 import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController;
33 import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
34 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
35 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSnapshotDoneRequest;
36 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSnapshotDoneResponse;
37 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SnapshotRequest;
38 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SnapshotResponse;
39 import org.junit.Test;
40 import org.junit.experimental.categories.Category;
41 import org.mockito.Mockito;
42
43 import com.google.protobuf.RpcController;
44
45
46
47
48 @Category(SmallTests.class)
49 public class TestSnapshotFromAdmin {
50
51 private static final Log LOG = LogFactory.getLog(TestSnapshotFromAdmin.class);
52
53
54
55
56
57
58 @Test(timeout = 60000)
59 public void testBackoffLogic() throws Exception {
60 final int pauseTime = 100;
61 final int maxWaitTime =
62 HConstants.RETRY_BACKOFF[HConstants.RETRY_BACKOFF.length - 1] * pauseTime;
63 final int numRetries = HConstants.RETRY_BACKOFF.length;
64
65
66 long ignoreExpectedTime = 0;
67 for (int i = 0; i < HConstants.RETRY_BACKOFF.length; i++) {
68 ignoreExpectedTime += HConstants.RETRY_BACKOFF[i] * pauseTime;
69 }
70
71 final long time = pauseTime * 3 + ((maxWaitTime / numRetries) * 3) + 300;
72 assertTrue("Capped snapshot wait time isn't less that the uncapped backoff time "
73 + "- further testing won't prove anything.", time < ignoreExpectedTime);
74
75
76 ConnectionManager.HConnectionImplementation mockConnection = Mockito
77 .mock(ConnectionManager.HConnectionImplementation.class);
78 Configuration conf = HBaseConfiguration.create();
79
80 conf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, numRetries);
81 conf.setLong("hbase.client.pause", pauseTime);
82
83
84 MasterKeepAliveConnection mockMaster = Mockito.mock(MasterKeepAliveConnection.class);
85 Mockito.when(mockConnection.getConfiguration()).thenReturn(conf);
86 Mockito.when(mockConnection.getKeepAliveMasterService()).thenReturn(mockMaster);
87
88 RpcRetryingCallerFactory callerFactory = new RpcRetryingCallerFactory(conf);
89 RpcControllerFactory controllerFactory = Mockito.mock(RpcControllerFactory.class);
90 Mockito.when(controllerFactory.newController()).thenReturn(
91 Mockito.mock(PayloadCarryingRpcController.class));
92 Mockito.when(mockConnection.getRpcRetryingCallerFactory()).thenReturn(callerFactory);
93 Mockito.when(mockConnection.getRpcControllerFactory()).thenReturn(controllerFactory);
94
95 SnapshotResponse response = SnapshotResponse.newBuilder()
96 .setExpectedTimeout(maxWaitTime)
97 .build();
98 Mockito
99 .when(
100 mockMaster.snapshot((RpcController) Mockito.any(),
101 Mockito.any(SnapshotRequest.class))).thenReturn(response);
102
103 IsSnapshotDoneResponse.Builder builder = IsSnapshotDoneResponse.newBuilder();
104 builder.setDone(false);
105
106 Mockito.when(
107 mockMaster.isSnapshotDone((RpcController) Mockito.any(),
108 Mockito.any(IsSnapshotDoneRequest.class))).thenReturn(builder.build(), builder.build(),
109 builder.build(), builder.build(), builder.build(), builder.setDone(true).build());
110
111
112 Admin admin = new HBaseAdmin(mockConnection);
113 String snapshot = "snapshot";
114 TableName table = TableName.valueOf("table");
115
116 long start = System.currentTimeMillis();
117 admin.snapshot(snapshot, table);
118 long finish = System.currentTimeMillis();
119 long elapsed = (finish - start);
120 assertTrue("Elapsed time:" + elapsed + " is more than expected max:" + time, elapsed <= time);
121 admin.close();
122 }
123
124
125
126
127
128
129 @Test
130 public void testValidateSnapshotName() throws Exception {
131 ConnectionManager.HConnectionImplementation mockConnection = Mockito
132 .mock(ConnectionManager.HConnectionImplementation.class);
133 Configuration conf = HBaseConfiguration.create();
134 Mockito.when(mockConnection.getConfiguration()).thenReturn(conf);
135
136 RpcRetryingCallerFactory callerFactory = new RpcRetryingCallerFactory(conf);
137 RpcControllerFactory controllerFactory = Mockito.mock(RpcControllerFactory.class);
138 Mockito.when(controllerFactory.newController()).thenReturn(
139 Mockito.mock(PayloadCarryingRpcController.class));
140 Mockito.when(mockConnection.getRpcRetryingCallerFactory()).thenReturn(callerFactory);
141 Mockito.when(mockConnection.getRpcControllerFactory()).thenReturn(controllerFactory);
142 Admin admin = new HBaseAdmin(mockConnection);
143 SnapshotDescription.Builder builder = SnapshotDescription.newBuilder();
144
145 failSnapshotStart(admin, builder.setName(HConstants.SNAPSHOT_DIR_NAME).build());
146 failSnapshotStart(admin, builder.setName("-snapshot").build());
147 failSnapshotStart(admin, builder.setName("snapshot fails").build());
148 failSnapshotStart(admin, builder.setName("snap$hot").build());
149 failSnapshotStart(admin, builder.setName("snap:hot").build());
150
151 failSnapshotStart(admin, builder.setName("snapshot").setTable(".table").build());
152 failSnapshotStart(admin, builder.setName("snapshot").setTable("-table").build());
153 failSnapshotStart(admin, builder.setName("snapshot").setTable("table fails").build());
154 failSnapshotStart(admin, builder.setName("snapshot").setTable("tab%le").build());
155
156
157 MasterKeepAliveConnection master = Mockito.mock(MasterKeepAliveConnection.class);
158 Mockito.when(mockConnection.getKeepAliveMasterService()).thenReturn(master);
159 SnapshotResponse response = SnapshotResponse.newBuilder().setExpectedTimeout(0).build();
160 Mockito.when(
161 master.snapshot((RpcController) Mockito.any(), Mockito.any(SnapshotRequest.class)))
162 .thenReturn(response);
163 IsSnapshotDoneResponse doneResponse = IsSnapshotDoneResponse.newBuilder().setDone(true).build();
164 Mockito.when(
165 master.isSnapshotDone((RpcController) Mockito.any(),
166 Mockito.any(IsSnapshotDoneRequest.class))).thenReturn(doneResponse);
167
168
169 admin.snapshot(builder.setName("snapshot").setTable("table").build());
170 }
171
172 private void failSnapshotStart(Admin admin, SnapshotDescription snapshot) throws IOException {
173 try {
174 admin.snapshot(snapshot);
175 fail("Snapshot should not have succeed with name:" + snapshot.getName());
176 } catch (IllegalArgumentException e) {
177 LOG.debug("Correctly failed to start snapshot:" + e.getMessage());
178 }
179 }
180 }