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 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
48
49 @Category(SmallTests.class)
50 public class TestSnapshotFromAdmin {
51
52 private static final Log LOG = LogFactory.getLog(TestSnapshotFromAdmin.class);
53
54
55
56
57
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
66
67 long ignoreExpectedTime = 0;
68 for (int i = 0; i < HConstants.RETRY_BACKOFF.length; i++) {
69 ignoreExpectedTime += HConstants.RETRY_BACKOFF[i] * pauseTime;
70 }
71
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
77 HConnectionManager.HConnectionImplementation mockConnection = Mockito
78 .mock(HConnectionManager.HConnectionImplementation.class);
79 Configuration conf = HBaseConfiguration.create();
80
81 conf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, numRetries);
82 conf.setLong("hbase.client.pause", pauseTime);
83
84 MasterAdminKeepAliveConnection mockMaster = Mockito.mock(MasterAdminKeepAliveConnection.class);
85 Mockito.when(mockConnection.getConfiguration()).thenReturn(conf);
86 Mockito.when(mockConnection.getKeepAliveMasterAdminService()).thenReturn(mockMaster);
87
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
96 IsSnapshotDoneResponse.Builder builder = IsSnapshotDoneResponse.newBuilder();
97 builder.setDone(false);
98
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
105 HBaseAdmin admin = new HBaseAdmin(mockConnection);
106 String snapshot = "snapshot";
107 TableName table = TableName.valueOf("table");
108
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
119
120
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
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
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
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
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 }