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 java.io.IOException;
023import java.util.Map;
024import java.util.regex.Pattern;
025import org.apache.hadoop.hbase.HConstants;
026import org.apache.hadoop.hbase.client.Connection;
027import org.apache.hadoop.hbase.client.Table;
028import org.apache.hadoop.hbase.client.coprocessor.Batch;
029import org.apache.hadoop.hbase.client.security.SecurityCapability;
030import org.apache.hadoop.hbase.ipc.CoprocessorRpcUtils;
031import org.apache.hadoop.hbase.ipc.ServerRpcController;
032import org.apache.hadoop.hbase.util.Bytes;
033import org.apache.yetus.audience.InterfaceAudience;
034
035import org.apache.hbase.thirdparty.com.google.protobuf.ByteString;
036import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException;
037import org.apache.hbase.thirdparty.com.google.protobuf.UnsafeByteOperations;
038
039import org.apache.hadoop.hbase.shaded.protobuf.generated.VisibilityLabelsProtos.GetAuthsRequest;
040import org.apache.hadoop.hbase.shaded.protobuf.generated.VisibilityLabelsProtos.GetAuthsResponse;
041import org.apache.hadoop.hbase.shaded.protobuf.generated.VisibilityLabelsProtos.ListLabelsRequest;
042import org.apache.hadoop.hbase.shaded.protobuf.generated.VisibilityLabelsProtos.ListLabelsResponse;
043import org.apache.hadoop.hbase.shaded.protobuf.generated.VisibilityLabelsProtos.SetAuthsRequest;
044import org.apache.hadoop.hbase.shaded.protobuf.generated.VisibilityLabelsProtos.VisibilityLabel;
045import org.apache.hadoop.hbase.shaded.protobuf.generated.VisibilityLabelsProtos.VisibilityLabelsRequest;
046import org.apache.hadoop.hbase.shaded.protobuf.generated.VisibilityLabelsProtos.VisibilityLabelsResponse;
047import org.apache.hadoop.hbase.shaded.protobuf.generated.VisibilityLabelsProtos.VisibilityLabelsService;
048
049/**
050 * Utility client for doing visibility labels admin operations.
051 */
052@InterfaceAudience.Public
053public class VisibilityClient {
054
055  /**
056   * Return true if cell visibility features are supported and enabled
057   * @param connection The connection to use
058   * @return true if cell visibility features are supported and enabled, false otherwise
059   */
060  public static boolean isCellVisibilityEnabled(Connection connection) throws IOException {
061    return connection.getAdmin().getSecurityCapabilities()
062      .contains(SecurityCapability.CELL_VISIBILITY);
063  }
064
065  /**
066   * Utility method for adding label to the system.
067   */
068  public static VisibilityLabelsResponse addLabel(Connection connection, final String label)
069    throws Throwable {
070    return addLabels(connection, new String[] { label });
071  }
072
073  /**
074   * Utility method for adding labels to the system.
075   */
076  public static VisibilityLabelsResponse addLabels(Connection connection, final String[] labels)
077    throws Throwable {
078    try (Table table = connection.getTable(LABELS_TABLE_NAME)) {
079      Batch.Call<VisibilityLabelsService, VisibilityLabelsResponse> callable =
080        new Batch.Call<VisibilityLabelsService, VisibilityLabelsResponse>() {
081          ServerRpcController controller = new ServerRpcController();
082          CoprocessorRpcUtils.BlockingRpcCallback<VisibilityLabelsResponse> rpcCallback =
083            new CoprocessorRpcUtils.BlockingRpcCallback<>();
084
085          @Override
086          public VisibilityLabelsResponse call(VisibilityLabelsService service) throws IOException {
087            VisibilityLabelsRequest.Builder builder = VisibilityLabelsRequest.newBuilder();
088            for (String label : labels) {
089              if (label.length() > 0) {
090                VisibilityLabel.Builder newBuilder = VisibilityLabel.newBuilder();
091                newBuilder.setLabel(UnsafeByteOperations.unsafeWrap(Bytes.toBytes(label)));
092                builder.addVisLabel(newBuilder.build());
093              }
094            }
095            service.addLabels(controller, builder.build(), rpcCallback);
096            VisibilityLabelsResponse response = rpcCallback.get();
097            if (controller.failedOnException()) {
098              throw controller.getFailedOn();
099            }
100            return response;
101          }
102        };
103      Map<byte[], VisibilityLabelsResponse> result =
104        table.coprocessorService(VisibilityLabelsService.class, HConstants.EMPTY_BYTE_ARRAY,
105          HConstants.EMPTY_BYTE_ARRAY, callable);
106      return result.values().iterator().next(); // There will be exactly one region for labels
107      // table and so one entry in result Map.
108    }
109  }
110
111  /**
112   * Sets given labels globally authorized for the user.
113   */
114  public static VisibilityLabelsResponse setAuths(Connection connection, final String[] auths,
115    final String user) throws Throwable {
116    return setOrClearAuths(connection, auths, user, true);
117  }
118
119  /**
120   * Get the authorization for a given user
121   * @param connection the Connection instance to use
122   * @param user       the user
123   * @return labels the given user is globally authorized for
124   */
125  public static GetAuthsResponse getAuths(Connection connection, final String user)
126    throws Throwable {
127    try (Table table = connection.getTable(LABELS_TABLE_NAME)) {
128      Batch.Call<VisibilityLabelsService, GetAuthsResponse> callable =
129        new Batch.Call<VisibilityLabelsService, GetAuthsResponse>() {
130          ServerRpcController controller = new ServerRpcController();
131          CoprocessorRpcUtils.BlockingRpcCallback<GetAuthsResponse> rpcCallback =
132            new CoprocessorRpcUtils.BlockingRpcCallback<>();
133
134          @Override
135          public GetAuthsResponse call(VisibilityLabelsService service) throws IOException {
136            GetAuthsRequest.Builder getAuthReqBuilder = GetAuthsRequest.newBuilder();
137            getAuthReqBuilder.setUser(UnsafeByteOperations.unsafeWrap(Bytes.toBytes(user)));
138            service.getAuths(controller, getAuthReqBuilder.build(), rpcCallback);
139            GetAuthsResponse response = rpcCallback.get();
140            if (controller.failedOnException()) {
141              throw controller.getFailedOn();
142            }
143            return response;
144          }
145        };
146      Map<byte[], GetAuthsResponse> result = table.coprocessorService(VisibilityLabelsService.class,
147        HConstants.EMPTY_BYTE_ARRAY, HConstants.EMPTY_BYTE_ARRAY, callable);
148      return result.values().iterator().next(); // There will be exactly one region for labels
149      // table and so one entry in result Map.
150    }
151  }
152
153  /**
154   * Retrieve the list of visibility labels defined in the system.
155   * @param connection The Connection instance to use.
156   * @param regex      The regular expression to filter which labels are returned.
157   * @return labels The list of visibility labels defined in the system.
158   */
159  public static ListLabelsResponse listLabels(Connection connection, final String regex)
160    throws Throwable {
161    try (Table table = connection.getTable(LABELS_TABLE_NAME)) {
162      Batch.Call<VisibilityLabelsService, ListLabelsResponse> callable =
163        new Batch.Call<VisibilityLabelsService, ListLabelsResponse>() {
164          ServerRpcController controller = new ServerRpcController();
165          CoprocessorRpcUtils.BlockingRpcCallback<ListLabelsResponse> rpcCallback =
166            new CoprocessorRpcUtils.BlockingRpcCallback<>();
167
168          @Override
169          public ListLabelsResponse call(VisibilityLabelsService service) throws IOException {
170            ListLabelsRequest.Builder listAuthLabelsReqBuilder = ListLabelsRequest.newBuilder();
171            if (regex != null) {
172              // Compile the regex here to catch any regex exception earlier.
173              Pattern pattern = Pattern.compile(regex);
174              listAuthLabelsReqBuilder.setRegex(pattern.toString());
175            }
176            service.listLabels(controller, listAuthLabelsReqBuilder.build(), rpcCallback);
177            ListLabelsResponse response = rpcCallback.get();
178            if (controller.failedOnException()) {
179              throw controller.getFailedOn();
180            }
181            return response;
182          }
183        };
184      Map<byte[], ListLabelsResponse> result =
185        table.coprocessorService(VisibilityLabelsService.class, HConstants.EMPTY_BYTE_ARRAY,
186          HConstants.EMPTY_BYTE_ARRAY, callable);
187      return result.values().iterator().next(); // There will be exactly one region for labels
188      // table and so one entry in result Map.
189    }
190  }
191
192  /**
193   * Removes given labels from user's globally authorized list of labels.
194   */
195  public static VisibilityLabelsResponse clearAuths(Connection connection, final String[] auths,
196    final String user) throws Throwable {
197    return setOrClearAuths(connection, auths, user, false);
198  }
199
200  private static VisibilityLabelsResponse setOrClearAuths(Connection connection,
201    final String[] auths, final String user, final boolean setOrClear)
202    throws IOException, ServiceException, Throwable {
203
204    try (Table table = connection.getTable(LABELS_TABLE_NAME)) {
205      Batch.Call<VisibilityLabelsService, VisibilityLabelsResponse> callable =
206        new Batch.Call<VisibilityLabelsService, VisibilityLabelsResponse>() {
207          ServerRpcController controller = new ServerRpcController();
208          CoprocessorRpcUtils.BlockingRpcCallback<VisibilityLabelsResponse> rpcCallback =
209            new CoprocessorRpcUtils.BlockingRpcCallback<>();
210
211          @Override
212          public VisibilityLabelsResponse call(VisibilityLabelsService service) throws IOException {
213            SetAuthsRequest.Builder setAuthReqBuilder = SetAuthsRequest.newBuilder();
214            setAuthReqBuilder.setUser(UnsafeByteOperations.unsafeWrap(Bytes.toBytes(user)));
215            for (String auth : auths) {
216              if (auth.length() > 0) {
217                setAuthReqBuilder.addAuth(ByteString.copyFromUtf8(auth));
218              }
219            }
220            if (setOrClear) {
221              service.setAuths(controller, setAuthReqBuilder.build(), rpcCallback);
222            } else {
223              service.clearAuths(controller, setAuthReqBuilder.build(), rpcCallback);
224            }
225            VisibilityLabelsResponse response = rpcCallback.get();
226            if (controller.failedOnException()) {
227              throw controller.getFailedOn();
228            }
229            return response;
230          }
231        };
232      Map<byte[], VisibilityLabelsResponse> result =
233        table.coprocessorService(VisibilityLabelsService.class, HConstants.EMPTY_BYTE_ARRAY,
234          HConstants.EMPTY_BYTE_ARRAY, callable);
235      return result.values().iterator().next(); // There will be exactly one region for labels
236      // table and so one entry in result Map.
237    }
238  }
239}