001/**
002 * Licensed to the Apache Software Foundation (ASF) under one
003 * or more contributor license agreements.  See the NOTICE file
004 * distributed with this work for additional information
005 * regarding copyright ownership.  The ASF licenses this file
006 * to you under the Apache License, Version 2.0 (the
007 * "License"); you may not use this file except in compliance
008 * with the License.  You may obtain a copy of the License at
009 *
010 * http://www.apache.org/licenses/LICENSE-2.0
011 *
012 * Unless required by applicable law or agreed to in writing, software
013 * distributed under the License is distributed on an "AS IS" BASIS,
014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
015 * See the License for the specific language governing permissions and
016 * limitations under the License.
017 */
018package org.apache.hadoop.hbase.security.visibility;
019
020import static org.apache.hadoop.hbase.security.visibility.VisibilityConstants.LABELS_TABLE_NAME;
021
022import com.google.protobuf.ByteString;
023import com.google.protobuf.ServiceException;
024
025import java.io.IOException;
026import java.util.Map;
027import java.util.regex.Pattern;
028
029import org.apache.hadoop.conf.Configuration;
030import org.apache.hadoop.hbase.HConstants;
031import org.apache.hadoop.hbase.client.Connection;
032import org.apache.hadoop.hbase.client.ConnectionFactory;
033import org.apache.hadoop.hbase.client.Table;
034import org.apache.hadoop.hbase.client.coprocessor.Batch;
035import org.apache.hadoop.hbase.client.security.SecurityCapability;
036import org.apache.hadoop.hbase.ipc.CoprocessorRpcUtils;
037import org.apache.hadoop.hbase.ipc.ServerRpcController;
038import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.GetAuthsRequest;
039import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.GetAuthsResponse;
040import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.ListLabelsRequest;
041import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.ListLabelsResponse;
042import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.SetAuthsRequest;
043import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.VisibilityLabel;
044import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.VisibilityLabelsRequest;
045import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.VisibilityLabelsResponse;
046import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.VisibilityLabelsService;
047import org.apache.hadoop.hbase.util.ByteStringer;
048import org.apache.hadoop.hbase.util.Bytes;
049import org.apache.yetus.audience.InterfaceAudience;
050
051
052/**
053 * Utility client for doing visibility labels admin operations.
054 */
055@InterfaceAudience.Public
056public class VisibilityClient {
057
058  /**
059   * Return true if cell visibility features are supported and enabled
060   * @param connection The connection to use
061   * @return true if cell visibility features are supported and enabled, false otherwise
062   * @throws IOException
063   */
064  public static boolean isCellVisibilityEnabled(Connection connection) throws IOException {
065    return connection.getAdmin().getSecurityCapabilities()
066        .contains(SecurityCapability.CELL_VISIBILITY);
067  }
068
069  /**
070   * Utility method for adding label to the system.
071   *
072   * @param conf
073   * @param label
074   * @return VisibilityLabelsResponse
075   * @throws Throwable
076   * @deprecated Use {@link #addLabel(Connection,String)} instead.
077   */
078  @Deprecated
079  public static VisibilityLabelsResponse addLabel(Configuration conf, final String label)
080      throws Throwable {
081    try (Connection connection = ConnectionFactory.createConnection(conf)) {
082      return addLabels(connection, new String[] { label });
083    }
084  }
085
086  /**
087   * Utility method for adding label to the system.
088   *
089   * @param connection
090   * @param label
091   * @return VisibilityLabelsResponse
092   * @throws Throwable
093   */
094  public static VisibilityLabelsResponse addLabel(Connection connection, final String label)
095      throws Throwable {
096    return addLabels(connection, new String[] { label });
097  }
098
099  /**
100   * Utility method for adding labels to the system.
101   *
102   * @param conf
103   * @param labels
104   * @return VisibilityLabelsResponse
105   * @throws Throwable
106   * @deprecated Use {@link #addLabels(Connection,String[])} instead.
107   */
108  @Deprecated
109  public static VisibilityLabelsResponse addLabels(Configuration conf, final String[] labels)
110      throws Throwable {
111    try (Connection connection = ConnectionFactory.createConnection(conf)) {
112      return addLabels(connection, labels);
113    }
114  }
115
116  /**
117   * Utility method for adding labels to the system.
118   *
119   * @param connection
120   * @param labels
121   * @return VisibilityLabelsResponse
122   * @throws Throwable
123   */
124  public static VisibilityLabelsResponse addLabels(Connection connection, final String[] labels)
125      throws Throwable {
126    try (Table table = connection.getTable(LABELS_TABLE_NAME)) {
127      Batch.Call<VisibilityLabelsService, VisibilityLabelsResponse> callable =
128          new Batch.Call<VisibilityLabelsService, VisibilityLabelsResponse>() {
129        ServerRpcController controller = new ServerRpcController();
130        CoprocessorRpcUtils.BlockingRpcCallback<VisibilityLabelsResponse> rpcCallback =
131            new CoprocessorRpcUtils.BlockingRpcCallback<>();
132
133        @Override
134        public VisibilityLabelsResponse call(VisibilityLabelsService service) throws IOException {
135          VisibilityLabelsRequest.Builder builder = VisibilityLabelsRequest.newBuilder();
136          for (String label : labels) {
137            if (label.length() > 0) {
138              VisibilityLabel.Builder newBuilder = VisibilityLabel.newBuilder();
139              newBuilder.setLabel(ByteStringer.wrap(Bytes.toBytes(label)));
140              builder.addVisLabel(newBuilder.build());
141            }
142          }
143          service.addLabels(controller, builder.build(), rpcCallback);
144          VisibilityLabelsResponse response = rpcCallback.get();
145          if (controller.failedOnException()) {
146            throw controller.getFailedOn();
147          }
148          return response;
149        }
150      };
151      Map<byte[], VisibilityLabelsResponse> result =
152          table.coprocessorService(VisibilityLabelsService.class, HConstants.EMPTY_BYTE_ARRAY,
153              HConstants.EMPTY_BYTE_ARRAY, callable);
154      return result.values().iterator().next(); // There will be exactly one region for labels
155      // table and so one entry in result Map.
156    }
157  }
158
159  /**
160   * Sets given labels globally authorized for the user.
161   * @param conf
162   * @param auths
163   * @param user
164   * @return VisibilityLabelsResponse
165   * @throws Throwable
166   * @deprecated Use {@link #setAuths(Connection,String[],String)} instead.
167   */
168  @Deprecated
169  public static VisibilityLabelsResponse setAuths(Configuration conf, final String[] auths,
170      final String user) throws Throwable {
171    try (Connection connection = ConnectionFactory.createConnection(conf)) {
172      return setOrClearAuths(connection, auths, user, true);
173    }
174  }
175
176  /**
177   * Sets given labels globally authorized for the user.
178   * @param connection
179   * @param auths
180   * @param user
181   * @return VisibilityLabelsResponse
182   * @throws Throwable
183   */
184  public static VisibilityLabelsResponse setAuths(Connection connection, final String[] auths,
185      final String user) throws Throwable {
186    return setOrClearAuths(connection, auths, user, true);
187  }
188
189  /**
190   * @param conf
191   * @param user
192   * @return labels, the given user is globally authorized for.
193   * @throws Throwable
194   * @deprecated Use {@link #getAuths(Connection,String)} instead.
195   */
196  @Deprecated
197  public static GetAuthsResponse getAuths(Configuration conf, final String user) throws Throwable {
198    try (Connection connection = ConnectionFactory.createConnection(conf)) {
199      return getAuths(connection, user);
200    }
201  }
202
203  /**
204   * @param connection the Connection instance to use.
205   * @param user
206   * @return labels, the given user is globally authorized for.
207   * @throws Throwable
208   */
209  public static GetAuthsResponse getAuths(Connection connection, final String user)
210      throws Throwable {
211    try (Table table = connection.getTable(LABELS_TABLE_NAME)) {
212      Batch.Call<VisibilityLabelsService, GetAuthsResponse> callable =
213          new Batch.Call<VisibilityLabelsService, GetAuthsResponse>() {
214        ServerRpcController controller = new ServerRpcController();
215        CoprocessorRpcUtils.BlockingRpcCallback<GetAuthsResponse> rpcCallback =
216            new CoprocessorRpcUtils.BlockingRpcCallback<>();
217
218        @Override
219        public GetAuthsResponse call(VisibilityLabelsService service) throws IOException {
220          GetAuthsRequest.Builder getAuthReqBuilder = GetAuthsRequest.newBuilder();
221          getAuthReqBuilder.setUser(ByteStringer.wrap(Bytes.toBytes(user)));
222          service.getAuths(controller, getAuthReqBuilder.build(), rpcCallback);
223          GetAuthsResponse response = rpcCallback.get();
224          if (controller.failedOnException()) {
225            throw controller.getFailedOn();
226          }
227          return response;
228        }
229      };
230      Map<byte[], GetAuthsResponse> result =
231          table.coprocessorService(VisibilityLabelsService.class, HConstants.EMPTY_BYTE_ARRAY,
232              HConstants.EMPTY_BYTE_ARRAY, callable);
233      return result.values().iterator().next(); // There will be exactly one region for labels
234      // table and so one entry in result Map.
235    }
236  }
237
238  /**
239   * Retrieve the list of visibility labels defined in the system.
240   * @param conf
241   * @param regex  The regular expression to filter which labels are returned.
242   * @return labels The list of visibility labels defined in the system.
243   * @throws Throwable
244   * @deprecated Use {@link #listLabels(Connection,String)} instead.
245   */
246  @Deprecated
247  public static ListLabelsResponse listLabels(Configuration conf, final String regex)
248      throws Throwable {
249    try(Connection connection = ConnectionFactory.createConnection(conf)){
250      return listLabels(connection, regex);
251    }
252  }
253
254  /**
255   * Retrieve the list of visibility labels defined in the system.
256   * @param connection The Connection instance to use.
257   * @param regex  The regular expression to filter which labels are returned.
258   * @return labels The list of visibility labels defined in the system.
259   * @throws Throwable
260   */
261  public static ListLabelsResponse listLabels(Connection connection, final String regex)
262      throws Throwable {
263    try (Table table = connection.getTable(LABELS_TABLE_NAME)) {
264      Batch.Call<VisibilityLabelsService, ListLabelsResponse> callable =
265          new Batch.Call<VisibilityLabelsService, ListLabelsResponse>() {
266        ServerRpcController controller = new ServerRpcController();
267        CoprocessorRpcUtils.BlockingRpcCallback<ListLabelsResponse> rpcCallback =
268            new CoprocessorRpcUtils.BlockingRpcCallback<>();
269
270        @Override
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        CoprocessorRpcUtils.BlockingRpcCallback<VisibilityLabelsResponse> rpcCallback =
333            new CoprocessorRpcUtils.BlockingRpcCallback<>();
334
335        @Override
336        public VisibilityLabelsResponse call(VisibilityLabelsService service) throws IOException {
337          SetAuthsRequest.Builder setAuthReqBuilder = SetAuthsRequest.newBuilder();
338          setAuthReqBuilder.setUser(ByteStringer.wrap(Bytes.toBytes(user)));
339          for (String auth : auths) {
340            if (auth.length() > 0) {
341              setAuthReqBuilder.addAuth((ByteString.copyFromUtf8(auth)));
342            }
343          }
344          if (setOrClear) {
345            service.setAuths(controller, setAuthReqBuilder.build(), rpcCallback);
346          } else {
347            service.clearAuths(controller, setAuthReqBuilder.build(), rpcCallback);
348          }
349          VisibilityLabelsResponse response = rpcCallback.get();
350          if (controller.failedOnException()) {
351            throw controller.getFailedOn();
352          }
353          return response;
354        }
355      };
356      Map<byte[], VisibilityLabelsResponse> result = table.coprocessorService(
357          VisibilityLabelsService.class, HConstants.EMPTY_BYTE_ARRAY, HConstants.EMPTY_BYTE_ARRAY,
358          callable);
359      return result.values().iterator().next(); // There will be exactly one region for labels
360      // table and so one entry in result Map.
361    }
362  }
363}