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.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   * Utility client for doing visibility labels admin operations.
53   */
54  @InterfaceAudience.Public
55  @InterfaceStability.Evolving
56  public class VisibilityClient {
57  
58    /**
59     * Utility method for adding label to the system.
60     *
61     * @param conf
62     * @param label
63     * @return VisibilityLabelsResponse
64     * @throws Throwable
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     * Utility method for adding labels to the system.
73     *
74     * @param conf
75     * @param labels
76     * @return VisibilityLabelsResponse
77     * @throws Throwable
78     */
79    public static VisibilityLabelsResponse addLabels(Configuration conf, final String[] labels)
80        throws Throwable {
81      // TODO: Make it so caller passes in a Connection rather than have us do this expensive
82      // setup each time.  This class only used in test and shell at moment though.
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(); // There will be exactly one region for labels
113         // table and so one entry in result Map.
114       }
115     }
116   }
117 
118   /**
119    * Sets given labels globally authorized for the user.
120    * @param conf
121    * @param auths
122    * @param user
123    * @return VisibilityLabelsResponse
124    * @throws Throwable
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    * @param conf
133    * @param user
134    * @return labels, the given user is globally authorized for.
135    * @throws Throwable
136    */
137   public static GetAuthsResponse getAuths(Configuration conf, final String user) throws Throwable {
138     // TODO: Make it so caller passes in a Connection rather than have us do this expensive
139     // setup each time.  This class only used in test and shell at moment though.
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(); // There will be exactly one region for labels
163         // table and so one entry in result Map.
164       }
165     }
166   }
167 
168   /**
169    * Retrieve the list of visibility labels defined in the system.
170    * @param conf
171    * @param regex  The regular expression to filter which labels are returned.
172    * @return labels The list of visibility labels defined in the system.
173    * @throws Throwable
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             // Compile the regex here to catch any regex exception earlier.
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(); // There will be exactly one region for labels
207       // table and so one entry in result Map.
208     }
209     finally {
210       if (table != null) {
211         table.close();
212       }
213       if (connection != null) {
214         connection.close();
215       }
216     }
217   }
218 
219   /**
220    * Removes given labels from user's globally authorized list of labels.
221    * @param conf
222    * @param auths
223    * @param user
224    * @return VisibilityLabelsResponse
225    * @throws Throwable
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     // TODO: Make it so caller passes in a Connection rather than have us do this expensive
235     // setup each time.  This class only used in test and shell at moment though.
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(); // There will be exactly one region for labels
268         // table and so one entry in result Map.
269       }
270     }
271   }
272 }