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             @Override
136             public VisibilityLabelsResponse call(VisibilityLabelsService service)
137                 throws IOException {
138               VisibilityLabelsRequest.Builder builder = VisibilityLabelsRequest.newBuilder();
139               for (String label : labels) {
140                 if (label.length() > 0) {
141                   VisibilityLabel.Builder newBuilder = VisibilityLabel.newBuilder();
142                   newBuilder.setLabel(ByteStringer.wrap(Bytes.toBytes(label)));
143                   builder.addVisLabel(newBuilder.build());
144                 }
145               }
146               service.addLabels(controller, builder.build(), rpcCallback);
147               VisibilityLabelsResponse response = rpcCallback.get();
148               if (controller.failedOnException()) {
149                 throw controller.getFailedOn();
150               }
151               return response;
152             }
153           };
154       Map<byte[], VisibilityLabelsResponse> result =
155           table.coprocessorService(VisibilityLabelsService.class, HConstants.EMPTY_BYTE_ARRAY,
156             HConstants.EMPTY_BYTE_ARRAY, callable);
157       return result.values().iterator().next(); // There will be exactly one region for labels
158       // table and so one entry in result Map.
159     }
160   }
161 
162   /**
163    * Sets given labels globally authorized for the user.
164    * @param conf
165    * @param auths
166    * @param user
167    * @return VisibilityLabelsResponse
168    * @throws Throwable
169    * @deprecated Use {@link #setAuths(Connection,String[],String)} instead.
170    */
171   @Deprecated
172   public static VisibilityLabelsResponse setAuths(Configuration conf, final String[] auths,
173       final String user) throws Throwable {
174     try (Connection connection = ConnectionFactory.createConnection(conf)) {
175       return setOrClearAuths(connection, auths, user, true);
176     }
177   }
178 
179   /**
180    * Sets given labels globally authorized for the user.
181    * @param connection
182    * @param auths
183    * @param user
184    * @return VisibilityLabelsResponse
185    * @throws Throwable
186    */
187   public static VisibilityLabelsResponse setAuths(Connection connection, final String[] auths,
188       final String user) throws Throwable {
189     return setOrClearAuths(connection, auths, user, true);
190   }
191 
192   /**
193    * @param conf
194    * @param user
195    * @return labels, the given user is globally authorized for.
196    * @throws Throwable
197    * @deprecated Use {@link #getAuths(Connection,String)} instead.
198    */
199   @Deprecated
200   public static GetAuthsResponse getAuths(Configuration conf, final String user) throws Throwable {
201     try (Connection connection = ConnectionFactory.createConnection(conf)) {
202       return getAuths(connection, user);
203     }
204   }
205 
206   /**
207    * @param connection the Connection instance to use.
208    * @param user
209    * @return labels, the given user is globally authorized for.
210    * @throws Throwable
211    */
212   public static GetAuthsResponse getAuths(Connection connection, final String user)
213       throws Throwable {
214       try (Table table = connection.getTable(LABELS_TABLE_NAME)) {
215         Batch.Call<VisibilityLabelsService, GetAuthsResponse> callable =
216             new Batch.Call<VisibilityLabelsService, GetAuthsResponse>() {
217           ServerRpcController controller = new ServerRpcController();
218           BlockingRpcCallback<GetAuthsResponse> rpcCallback =
219               new BlockingRpcCallback<GetAuthsResponse>();
220 
221           @Override
222           public GetAuthsResponse call(VisibilityLabelsService service) throws IOException {
223             GetAuthsRequest.Builder getAuthReqBuilder = GetAuthsRequest.newBuilder();
224             getAuthReqBuilder.setUser(ByteStringer.wrap(Bytes.toBytes(user)));
225             service.getAuths(controller, getAuthReqBuilder.build(), rpcCallback);
226             GetAuthsResponse response = rpcCallback.get();
227             if (controller.failedOnException()) {
228               throw controller.getFailedOn();
229             }
230             return response;
231           }
232         };
233         Map<byte[], GetAuthsResponse> result =
234           table.coprocessorService(VisibilityLabelsService.class,
235             HConstants.EMPTY_BYTE_ARRAY, HConstants.EMPTY_BYTE_ARRAY, callable);
236         return result.values().iterator().next(); // There will be exactly one region for labels
237         // table and so one entry in result Map.
238       }
239   }
240 
241   /**
242    * Retrieve the list of visibility labels defined in the system.
243    * @param conf
244    * @param regex  The regular expression to filter which labels are returned.
245    * @return labels The list of visibility labels defined in the system.
246    * @throws Throwable
247    * @deprecated Use {@link #listLabels(Connection,String)} instead.
248    */
249   @Deprecated
250   public static ListLabelsResponse listLabels(Configuration conf, final String regex)
251       throws Throwable {
252     try(Connection connection = ConnectionFactory.createConnection(conf)){
253       return listLabels(connection, regex);
254     }
255   }
256 
257   /**
258    * Retrieve the list of visibility labels defined in the system.
259    * @param connection The Connection instance to use.
260    * @param regex  The regular expression to filter which labels are returned.
261    * @return labels The list of visibility labels defined in the system.
262    * @throws Throwable
263    */
264   public static ListLabelsResponse listLabels(Connection connection, final String regex)
265       throws Throwable {
266     try (Table table = connection.getTable(LABELS_TABLE_NAME)) {
267       Batch.Call<VisibilityLabelsService, ListLabelsResponse> callable =
268           new Batch.Call<VisibilityLabelsService, ListLabelsResponse>() {
269             ServerRpcController controller = new ServerRpcController();
270             BlockingRpcCallback<ListLabelsResponse> rpcCallback =
271                 new BlockingRpcCallback<ListLabelsResponse>();
272 
273             @Override
274             public ListLabelsResponse call(VisibilityLabelsService service) throws IOException {
275               ListLabelsRequest.Builder listAuthLabelsReqBuilder = ListLabelsRequest.newBuilder();
276               if (regex != null) {
277                 // Compile the regex here to catch any regex exception earlier.
278                 Pattern pattern = Pattern.compile(regex);
279                 listAuthLabelsReqBuilder.setRegex(pattern.toString());
280               }
281               service.listLabels(controller, listAuthLabelsReqBuilder.build(), rpcCallback);
282               ListLabelsResponse response = rpcCallback.get();
283               if (controller.failedOnException()) {
284                 throw controller.getFailedOn();
285               }
286               return response;
287             }
288           };
289       Map<byte[], ListLabelsResponse> result =
290           table.coprocessorService(VisibilityLabelsService.class, HConstants.EMPTY_BYTE_ARRAY,
291             HConstants.EMPTY_BYTE_ARRAY, callable);
292       return result.values().iterator().next(); // There will be exactly one region for labels
293       // table and so one entry in result Map.
294     }
295   }
296 
297   /**
298    * Removes given labels from user's globally authorized list of labels.
299    * @param conf
300    * @param auths
301    * @param user
302    * @return VisibilityLabelsResponse
303    * @throws Throwable
304    * @deprecated Use {@link #clearAuths(Connection,String[],String)} instead.
305    */
306   @Deprecated
307   public static VisibilityLabelsResponse clearAuths(Configuration conf, final String[] auths,
308       final String user) throws Throwable {
309     try (Connection connection = ConnectionFactory.createConnection(conf)) {
310       return setOrClearAuths(connection, auths, user, false);
311     }
312   }
313 
314   /**
315    * Removes given labels from user's globally authorized list of labels.
316    * @param connection
317    * @param auths
318    * @param user
319    * @return VisibilityLabelsResponse
320    * @throws Throwable
321    */
322   public static VisibilityLabelsResponse clearAuths(Connection connection, final String[] auths,
323       final String user) throws Throwable {
324     return setOrClearAuths(connection, auths, user, false);
325   }
326 
327   private static VisibilityLabelsResponse setOrClearAuths(Connection connection,
328       final String[] auths, final String user, final boolean setOrClear)
329           throws IOException, ServiceException, Throwable {
330 
331       try (Table table = connection.getTable(LABELS_TABLE_NAME)) {
332         Batch.Call<VisibilityLabelsService, VisibilityLabelsResponse> callable =
333             new Batch.Call<VisibilityLabelsService, VisibilityLabelsResponse>() {
334           ServerRpcController controller = new ServerRpcController();
335           BlockingRpcCallback<VisibilityLabelsResponse> rpcCallback =
336               new BlockingRpcCallback<VisibilityLabelsResponse>();
337 
338           @Override
339           public VisibilityLabelsResponse call(VisibilityLabelsService service) throws IOException {
340             SetAuthsRequest.Builder setAuthReqBuilder = SetAuthsRequest.newBuilder();
341             setAuthReqBuilder.setUser(ByteStringer.wrap(Bytes.toBytes(user)));
342             for (String auth : auths) {
343               if (auth.length() > 0) {
344                 setAuthReqBuilder.addAuth(ByteStringer.wrap(Bytes.toBytes(auth)));
345               }
346             }
347             if (setOrClear) {
348               service.setAuths(controller, setAuthReqBuilder.build(), rpcCallback);
349             } else {
350               service.clearAuths(controller, setAuthReqBuilder.build(), rpcCallback);
351             }
352             VisibilityLabelsResponse response = rpcCallback.get();
353             if (controller.failedOnException()) {
354               throw controller.getFailedOn();
355             }
356             return response;
357           }
358         };
359         Map<byte[], VisibilityLabelsResponse> result = table.coprocessorService(
360             VisibilityLabelsService.class, HConstants.EMPTY_BYTE_ARRAY, HConstants.EMPTY_BYTE_ARRAY,
361             callable);
362         return result.values().iterator().next(); // There will be exactly one region for labels
363         // table and so one entry in result Map.
364       }
365   }
366 }