View Javadoc

1   /**
2    * Licensed to the Apache Software Foundation (ASF) under one
3    * or more contributor license agreements.  See the NOTICE file
4    * distributed with this work for additional information
5    * regarding copyright ownership.  The ASF licenses this file
6    * to you under the Apache License, Version 2.0 (the
7    * "License"); you may not use this file except in compliance
8    * with the License.  You may obtain a copy of the License at
9    *
10   * http://www.apache.org/licenses/LICENSE-2.0
11   *
12   * Unless required by applicable law or agreed to in writing, software
13   * distributed under the License is distributed on an "AS IS" BASIS,
14   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15   * See the License for the specific language governing permissions and
16   * limitations under the License.
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   * Utility client for doing visibility labels admin operations.
54   */
55  @InterfaceAudience.Public
56  @InterfaceStability.Evolving
57  public class VisibilityClient {
58  
59    /**
60     * Return true if cell visibility features are supported and enabled
61     * @param connection The connection to use
62     * @return true if cell visibility features are supported and enabled, false otherwise
63     * @throws IOException
64     */
65    public static boolean isCellVisibilityEnabled(Connection connection) throws IOException {
66      return connection.getAdmin().getSecurityCapabilities()
67          .contains(SecurityCapability.CELL_VISIBILITY);
68    }
69  
70    /**
71     * Utility method for adding label to the system.
72     *
73     * @param conf
74     * @param label
75     * @return VisibilityLabelsResponse
76     * @throws Throwable
77     * @deprecated Use {@link #addLabel(Connection,String)} instead.
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     * Utility method for adding label to the system.
89     *
90     * @param connection
91     * @param label
92     * @return VisibilityLabelsResponse
93     * @throws Throwable
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    * Utility method for adding labels to the system.
102    *
103    * @param conf
104    * @param labels
105    * @return VisibilityLabelsResponse
106    * @throws Throwable
107    * @deprecated Use {@link #addLabels(Connection,String[])} instead.
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    * Utility method for adding labels to the system.
119    *
120    * @param connection
121    * @param labels
122    * @return VisibilityLabelsResponse
123    * @throws Throwable
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(); // There will be exactly one region for labels
157       // table and so one entry in result Map.
158     }
159   }
160 
161   /**
162    * Sets given labels globally authorized for the user.
163    * @param conf
164    * @param auths
165    * @param user
166    * @return VisibilityLabelsResponse
167    * @throws Throwable
168    * @deprecated Use {@link #setAuths(Connection,String[],String)} instead.
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    * Sets given labels globally authorized for the user.
180    * @param connection
181    * @param auths
182    * @param user
183    * @return VisibilityLabelsResponse
184    * @throws Throwable
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    * @param conf
193    * @param user
194    * @return labels, the given user is globally authorized for.
195    * @throws Throwable
196    * @deprecated Use {@link #getAuths(Connection,String)} instead.
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    * @param connection the Connection instance to use.
207    * @param user
208    * @return labels, the given user is globally authorized for.
209    * @throws Throwable
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(); // There will be exactly one region for labels
235         // table and so one entry in result Map.
236       }
237   }
238 
239   /**
240    * Retrieve the list of visibility labels defined in the system.
241    * @param conf
242    * @param regex  The regular expression to filter which labels are returned.
243    * @return labels The list of visibility labels defined in the system.
244    * @throws Throwable
245    * @deprecated Use {@link #listLabels(Connection,String)} instead.
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    * Retrieve the list of visibility labels defined in the system.
257    * @param connection The Connection instance to use.
258    * @param regex  The regular expression to filter which labels are returned.
259    * @return labels The list of visibility labels defined in the system.
260    * @throws Throwable
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                 // Compile the regex here to catch any regex exception earlier.
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(); // There will be exactly one region for labels
290       // table and so one entry in result Map.
291     }
292   }
293 
294   /**
295    * Removes given labels from user's globally authorized list of labels.
296    * @param conf
297    * @param auths
298    * @param user
299    * @return VisibilityLabelsResponse
300    * @throws Throwable
301    * @deprecated Use {@link #clearAuths(Connection,String[],String)} instead.
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    * Removes given labels from user's globally authorized list of labels.
313    * @param connection
314    * @param auths
315    * @param user
316    * @return VisibilityLabelsResponse
317    * @throws Throwable
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(); // There will be exactly one region for labels
359         // table and so one entry in result Map.
360       }
361   }
362 }