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.ipc.BlockingRpcCallback;
37 import org.apache.hadoop.hbase.ipc.ServerRpcController;
38 import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.GetAuthsRequest;
39 import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.GetAuthsResponse;
40 import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.ListLabelsRequest;
41 import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.ListLabelsResponse;
42 import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.SetAuthsRequest;
43 import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.VisibilityLabel;
44 import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.VisibilityLabelsRequest;
45 import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.VisibilityLabelsResponse;
46 import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.VisibilityLabelsService;
47 import org.apache.hadoop.hbase.util.Bytes;
48
49 import com.google.protobuf.ServiceException;
50
51
52
53
54 @InterfaceAudience.Public
55 @InterfaceStability.Evolving
56 public class VisibilityClient {
57
58
59
60
61
62
63
64
65
66 public static VisibilityLabelsResponse addLabel(Configuration conf, final String label)
67 throws Throwable {
68 return addLabels(conf, new String[] { label });
69 }
70
71
72
73
74
75
76
77
78
79 public static VisibilityLabelsResponse addLabels(Configuration conf, final String[] labels)
80 throws Throwable {
81
82
83 try (Connection connection = ConnectionFactory.createConnection(conf)) {
84 try (Table table = connection.getTable(LABELS_TABLE_NAME)) {
85 Batch.Call<VisibilityLabelsService, VisibilityLabelsResponse> callable =
86 new Batch.Call<VisibilityLabelsService, VisibilityLabelsResponse>() {
87 ServerRpcController controller = new ServerRpcController();
88 BlockingRpcCallback<VisibilityLabelsResponse> rpcCallback =
89 new BlockingRpcCallback<VisibilityLabelsResponse>();
90
91 public VisibilityLabelsResponse call(VisibilityLabelsService service)
92 throws IOException {
93 VisibilityLabelsRequest.Builder builder = VisibilityLabelsRequest.newBuilder();
94 for (String label : labels) {
95 if (label.length() > 0) {
96 VisibilityLabel.Builder newBuilder = VisibilityLabel.newBuilder();
97 newBuilder.setLabel(ByteStringer.wrap(Bytes.toBytes(label)));
98 builder.addVisLabel(newBuilder.build());
99 }
100 }
101 service.addLabels(controller, builder.build(), rpcCallback);
102 VisibilityLabelsResponse response = rpcCallback.get();
103 if (controller.failedOnException()) {
104 throw controller.getFailedOn();
105 }
106 return response;
107 }
108 };
109 Map<byte[], VisibilityLabelsResponse> result =
110 table.coprocessorService(VisibilityLabelsService.class, HConstants.EMPTY_BYTE_ARRAY,
111 HConstants.EMPTY_BYTE_ARRAY, callable);
112 return result.values().iterator().next();
113
114 }
115 }
116 }
117
118
119
120
121
122
123
124
125
126 public static VisibilityLabelsResponse setAuths(Configuration conf, final String[] auths,
127 final String user) throws Throwable {
128 return setOrClearAuths(conf, auths, user, true);
129 }
130
131
132
133
134
135
136
137 public static GetAuthsResponse getAuths(Configuration conf, final String user) throws Throwable {
138
139
140 try (Connection connection = ConnectionFactory.createConnection(conf)) {
141 try (Table table = connection.getTable(LABELS_TABLE_NAME)) {
142 Batch.Call<VisibilityLabelsService, GetAuthsResponse> callable =
143 new Batch.Call<VisibilityLabelsService, GetAuthsResponse>() {
144 ServerRpcController controller = new ServerRpcController();
145 BlockingRpcCallback<GetAuthsResponse> rpcCallback =
146 new BlockingRpcCallback<GetAuthsResponse>();
147
148 public GetAuthsResponse call(VisibilityLabelsService service) throws IOException {
149 GetAuthsRequest.Builder getAuthReqBuilder = GetAuthsRequest.newBuilder();
150 getAuthReqBuilder.setUser(ByteStringer.wrap(Bytes.toBytes(user)));
151 service.getAuths(controller, getAuthReqBuilder.build(), rpcCallback);
152 GetAuthsResponse response = rpcCallback.get();
153 if (controller.failedOnException()) {
154 throw controller.getFailedOn();
155 }
156 return response;
157 }
158 };
159 Map<byte[], GetAuthsResponse> result =
160 table.coprocessorService(VisibilityLabelsService.class,
161 HConstants.EMPTY_BYTE_ARRAY, HConstants.EMPTY_BYTE_ARRAY, callable);
162 return result.values().iterator().next();
163
164 }
165 }
166 }
167
168
169
170
171
172
173
174
175 public static ListLabelsResponse listLabels(Configuration conf, final String regex)
176 throws Throwable {
177 Connection connection = null;
178 Table table = null;
179 try {
180 connection = ConnectionFactory.createConnection(conf);
181 table = connection.getTable(LABELS_TABLE_NAME);
182 Batch.Call<VisibilityLabelsService, ListLabelsResponse> callable =
183 new Batch.Call<VisibilityLabelsService, ListLabelsResponse>() {
184 ServerRpcController controller = new ServerRpcController();
185 BlockingRpcCallback<ListLabelsResponse> rpcCallback =
186 new BlockingRpcCallback<ListLabelsResponse>();
187
188 public ListLabelsResponse call(VisibilityLabelsService service) throws IOException {
189 ListLabelsRequest.Builder listAuthLabelsReqBuilder = ListLabelsRequest.newBuilder();
190 if (regex != null) {
191
192 Pattern pattern = Pattern.compile(regex);
193 listAuthLabelsReqBuilder.setRegex(pattern.toString());
194 }
195 service.listLabels(controller, listAuthLabelsReqBuilder.build(), rpcCallback);
196 ListLabelsResponse response = rpcCallback.get();
197 if (controller.failedOnException()) {
198 throw controller.getFailedOn();
199 }
200 return response;
201 }
202 };
203 Map<byte[], ListLabelsResponse> result =
204 table.coprocessorService(VisibilityLabelsService.class, HConstants.EMPTY_BYTE_ARRAY,
205 HConstants.EMPTY_BYTE_ARRAY, callable);
206 return result.values().iterator().next();
207
208 }
209 finally {
210 if (table != null) {
211 table.close();
212 }
213 if (connection != null) {
214 connection.close();
215 }
216 }
217 }
218
219
220
221
222
223
224
225
226
227 public static VisibilityLabelsResponse clearAuths(Configuration conf, final String[] auths,
228 final String user) throws Throwable {
229 return setOrClearAuths(conf, auths, user, false);
230 }
231
232 private static VisibilityLabelsResponse setOrClearAuths(Configuration conf, final String[] auths,
233 final String user, final boolean setOrClear) throws IOException, ServiceException, Throwable {
234
235
236 try (Connection connection = ConnectionFactory.createConnection(conf)) {
237 try (Table table = connection.getTable(LABELS_TABLE_NAME)) {
238 Batch.Call<VisibilityLabelsService, VisibilityLabelsResponse> callable =
239 new Batch.Call<VisibilityLabelsService, VisibilityLabelsResponse>() {
240 ServerRpcController controller = new ServerRpcController();
241 BlockingRpcCallback<VisibilityLabelsResponse> rpcCallback =
242 new BlockingRpcCallback<VisibilityLabelsResponse>();
243
244 public VisibilityLabelsResponse call(VisibilityLabelsService service) throws IOException {
245 SetAuthsRequest.Builder setAuthReqBuilder = SetAuthsRequest.newBuilder();
246 setAuthReqBuilder.setUser(ByteStringer.wrap(Bytes.toBytes(user)));
247 for (String auth : auths) {
248 if (auth.length() > 0) {
249 setAuthReqBuilder.addAuth(ByteStringer.wrap(Bytes.toBytes(auth)));
250 }
251 }
252 if (setOrClear) {
253 service.setAuths(controller, setAuthReqBuilder.build(), rpcCallback);
254 } else {
255 service.clearAuths(controller, setAuthReqBuilder.build(), rpcCallback);
256 }
257 VisibilityLabelsResponse response = rpcCallback.get();
258 if (controller.failedOnException()) {
259 throw controller.getFailedOn();
260 }
261 return response;
262 }
263 };
264 Map<byte[], VisibilityLabelsResponse> result = table.coprocessorService(
265 VisibilityLabelsService.class, HConstants.EMPTY_BYTE_ARRAY, HConstants.EMPTY_BYTE_ARRAY,
266 callable);
267 return result.values().iterator().next();
268
269 }
270 }
271 }
272 }