1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18 package org.apache.hadoop.hbase.security.visibility;
19
20 import static org.apache.hadoop.hbase.security.visibility.VisibilityConstants.LABELS_TABLE_NAME;
21
22 import java.io.IOException;
23 import java.util.Map;
24 import java.util.regex.Pattern;
25
26 import org.apache.hadoop.conf.Configuration;
27 import org.apache.hadoop.hbase.HConstants;
28 import org.apache.hadoop.hbase.client.Table;
29 import org.apache.hadoop.hbase.util.ByteStringer;
30 import org.apache.hadoop.hbase.classification.InterfaceAudience;
31 import org.apache.hadoop.hbase.classification.InterfaceStability;
32 import org.apache.hadoop.hbase.client.Connection;
33 import org.apache.hadoop.hbase.client.ConnectionFactory;
34 import org.apache.hadoop.hbase.client.Table;
35 import org.apache.hadoop.hbase.client.coprocessor.Batch;
36 import org.apache.hadoop.hbase.client.security.SecurityCapability;
37 import org.apache.hadoop.hbase.ipc.BlockingRpcCallback;
38 import org.apache.hadoop.hbase.ipc.ServerRpcController;
39 import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.GetAuthsRequest;
40 import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.GetAuthsResponse;
41 import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.ListLabelsRequest;
42 import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.ListLabelsResponse;
43 import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.SetAuthsRequest;
44 import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.VisibilityLabel;
45 import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.VisibilityLabelsRequest;
46 import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.VisibilityLabelsResponse;
47 import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.VisibilityLabelsService;
48 import org.apache.hadoop.hbase.util.Bytes;
49
50 import com.google.protobuf.ServiceException;
51
52
53
54
55 @InterfaceAudience.Public
56 @InterfaceStability.Evolving
57 public class VisibilityClient {
58
59
60
61
62
63
64
65 public static boolean isCellVisibilityEnabled(Connection connection) throws IOException {
66 return connection.getAdmin().getSecurityCapabilities()
67 .contains(SecurityCapability.CELL_VISIBILITY);
68 }
69
70
71
72
73
74
75
76
77
78
79 @Deprecated
80 public static VisibilityLabelsResponse addLabel(Configuration conf, final String label)
81 throws Throwable {
82 try (Connection connection = ConnectionFactory.createConnection(conf)) {
83 return addLabels(connection, new String[] { label });
84 }
85 }
86
87
88
89
90
91
92
93
94
95 public static VisibilityLabelsResponse addLabel(Connection connection, final String label)
96 throws Throwable {
97 return addLabels(connection, new String[] { label });
98 }
99
100
101
102
103
104
105
106
107
108
109 @Deprecated
110 public static VisibilityLabelsResponse addLabels(Configuration conf, final String[] labels)
111 throws Throwable {
112 try (Connection connection = ConnectionFactory.createConnection(conf)) {
113 return addLabels(connection, labels);
114 }
115 }
116
117
118
119
120
121
122
123
124
125 public static VisibilityLabelsResponse addLabels(Connection connection, final String[] labels)
126 throws Throwable {
127
128 try (Table table = connection.getTable(LABELS_TABLE_NAME)) {
129 Batch.Call<VisibilityLabelsService, VisibilityLabelsResponse> callable =
130 new Batch.Call<VisibilityLabelsService, VisibilityLabelsResponse>() {
131 ServerRpcController controller = new ServerRpcController();
132 BlockingRpcCallback<VisibilityLabelsResponse> rpcCallback =
133 new BlockingRpcCallback<VisibilityLabelsResponse>();
134
135 public VisibilityLabelsResponse call(VisibilityLabelsService service)
136 throws IOException {
137 VisibilityLabelsRequest.Builder builder = VisibilityLabelsRequest.newBuilder();
138 for (String label : labels) {
139 if (label.length() > 0) {
140 VisibilityLabel.Builder newBuilder = VisibilityLabel.newBuilder();
141 newBuilder.setLabel(ByteStringer.wrap(Bytes.toBytes(label)));
142 builder.addVisLabel(newBuilder.build());
143 }
144 }
145 service.addLabels(controller, builder.build(), rpcCallback);
146 VisibilityLabelsResponse response = rpcCallback.get();
147 if (controller.failedOnException()) {
148 throw controller.getFailedOn();
149 }
150 return response;
151 }
152 };
153 Map<byte[], VisibilityLabelsResponse> result =
154 table.coprocessorService(VisibilityLabelsService.class, HConstants.EMPTY_BYTE_ARRAY,
155 HConstants.EMPTY_BYTE_ARRAY, callable);
156 return result.values().iterator().next();
157
158 }
159 }
160
161
162
163
164
165
166
167
168
169
170 @Deprecated
171 public static VisibilityLabelsResponse setAuths(Configuration conf, final String[] auths,
172 final String user) throws Throwable {
173 try (Connection connection = ConnectionFactory.createConnection(conf)) {
174 return setOrClearAuths(connection, auths, user, true);
175 }
176 }
177
178
179
180
181
182
183
184
185
186 public static VisibilityLabelsResponse setAuths(Connection connection, final String[] auths,
187 final String user) throws Throwable {
188 return setOrClearAuths(connection, auths, user, true);
189 }
190
191
192
193
194
195
196
197
198 @Deprecated
199 public static GetAuthsResponse getAuths(Configuration conf, final String user) throws Throwable {
200 try (Connection connection = ConnectionFactory.createConnection(conf)) {
201 return getAuths(connection, user);
202 }
203 }
204
205
206
207
208
209
210
211 public static GetAuthsResponse getAuths(Connection connection, final String user)
212 throws Throwable {
213 try (Table table = connection.getTable(LABELS_TABLE_NAME)) {
214 Batch.Call<VisibilityLabelsService, GetAuthsResponse> callable =
215 new Batch.Call<VisibilityLabelsService, GetAuthsResponse>() {
216 ServerRpcController controller = new ServerRpcController();
217 BlockingRpcCallback<GetAuthsResponse> rpcCallback =
218 new BlockingRpcCallback<GetAuthsResponse>();
219
220 public GetAuthsResponse call(VisibilityLabelsService service) throws IOException {
221 GetAuthsRequest.Builder getAuthReqBuilder = GetAuthsRequest.newBuilder();
222 getAuthReqBuilder.setUser(ByteStringer.wrap(Bytes.toBytes(user)));
223 service.getAuths(controller, getAuthReqBuilder.build(), rpcCallback);
224 GetAuthsResponse response = rpcCallback.get();
225 if (controller.failedOnException()) {
226 throw controller.getFailedOn();
227 }
228 return response;
229 }
230 };
231 Map<byte[], GetAuthsResponse> result =
232 table.coprocessorService(VisibilityLabelsService.class,
233 HConstants.EMPTY_BYTE_ARRAY, HConstants.EMPTY_BYTE_ARRAY, callable);
234 return result.values().iterator().next();
235
236 }
237 }
238
239
240
241
242
243
244
245
246
247 @Deprecated
248 public static ListLabelsResponse listLabels(Configuration conf, final String regex)
249 throws Throwable {
250 try(Connection connection = ConnectionFactory.createConnection(conf)){
251 return listLabels(connection, regex);
252 }
253 }
254
255
256
257
258
259
260
261
262 public static ListLabelsResponse listLabels(Connection connection, final String regex)
263 throws Throwable {
264 try (Table table = connection.getTable(LABELS_TABLE_NAME)) {
265 Batch.Call<VisibilityLabelsService, ListLabelsResponse> callable =
266 new Batch.Call<VisibilityLabelsService, ListLabelsResponse>() {
267 ServerRpcController controller = new ServerRpcController();
268 BlockingRpcCallback<ListLabelsResponse> rpcCallback =
269 new BlockingRpcCallback<ListLabelsResponse>();
270
271 public ListLabelsResponse call(VisibilityLabelsService service) throws IOException {
272 ListLabelsRequest.Builder listAuthLabelsReqBuilder = ListLabelsRequest.newBuilder();
273 if (regex != null) {
274
275 Pattern pattern = Pattern.compile(regex);
276 listAuthLabelsReqBuilder.setRegex(pattern.toString());
277 }
278 service.listLabels(controller, listAuthLabelsReqBuilder.build(), rpcCallback);
279 ListLabelsResponse response = rpcCallback.get();
280 if (controller.failedOnException()) {
281 throw controller.getFailedOn();
282 }
283 return response;
284 }
285 };
286 Map<byte[], ListLabelsResponse> result =
287 table.coprocessorService(VisibilityLabelsService.class, HConstants.EMPTY_BYTE_ARRAY,
288 HConstants.EMPTY_BYTE_ARRAY, callable);
289 return result.values().iterator().next();
290
291 }
292 }
293
294
295
296
297
298
299
300
301
302
303 @Deprecated
304 public static VisibilityLabelsResponse clearAuths(Configuration conf, final String[] auths,
305 final String user) throws Throwable {
306 try (Connection connection = ConnectionFactory.createConnection(conf)) {
307 return setOrClearAuths(connection, auths, user, false);
308 }
309 }
310
311
312
313
314
315
316
317
318
319 public static VisibilityLabelsResponse clearAuths(Connection connection, final String[] auths,
320 final String user) throws Throwable {
321 return setOrClearAuths(connection, auths, user, false);
322 }
323
324 private static VisibilityLabelsResponse setOrClearAuths(Connection connection,
325 final String[] auths, final String user, final boolean setOrClear)
326 throws IOException, ServiceException, Throwable {
327
328 try (Table table = connection.getTable(LABELS_TABLE_NAME)) {
329 Batch.Call<VisibilityLabelsService, VisibilityLabelsResponse> callable =
330 new Batch.Call<VisibilityLabelsService, VisibilityLabelsResponse>() {
331 ServerRpcController controller = new ServerRpcController();
332 BlockingRpcCallback<VisibilityLabelsResponse> rpcCallback =
333 new BlockingRpcCallback<VisibilityLabelsResponse>();
334
335 public VisibilityLabelsResponse call(VisibilityLabelsService service) throws IOException {
336 SetAuthsRequest.Builder setAuthReqBuilder = SetAuthsRequest.newBuilder();
337 setAuthReqBuilder.setUser(ByteStringer.wrap(Bytes.toBytes(user)));
338 for (String auth : auths) {
339 if (auth.length() > 0) {
340 setAuthReqBuilder.addAuth(ByteStringer.wrap(Bytes.toBytes(auth)));
341 }
342 }
343 if (setOrClear) {
344 service.setAuths(controller, setAuthReqBuilder.build(), rpcCallback);
345 } else {
346 service.clearAuths(controller, setAuthReqBuilder.build(), rpcCallback);
347 }
348 VisibilityLabelsResponse response = rpcCallback.get();
349 if (controller.failedOnException()) {
350 throw controller.getFailedOn();
351 }
352 return response;
353 }
354 };
355 Map<byte[], VisibilityLabelsResponse> result = table.coprocessorService(
356 VisibilityLabelsService.class, HConstants.EMPTY_BYTE_ARRAY, HConstants.EMPTY_BYTE_ARRAY,
357 callable);
358 return result.values().iterator().next();
359
360 }
361 }
362 }