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;
024import java.io.IOException;
025import java.util.Map;
026import java.util.regex.Pattern;
027import org.apache.hadoop.conf.Configuration;
028import org.apache.hadoop.hbase.HConstants;
029import org.apache.hadoop.hbase.client.Connection;
030import org.apache.hadoop.hbase.client.ConnectionFactory;
031import org.apache.hadoop.hbase.client.Table;
032import org.apache.hadoop.hbase.client.coprocessor.Batch;
033import org.apache.hadoop.hbase.client.security.SecurityCapability;
034import org.apache.hadoop.hbase.ipc.CoprocessorRpcUtils;
035import org.apache.hadoop.hbase.ipc.ServerRpcController;
036import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.GetAuthsRequest;
037import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.GetAuthsResponse;
038import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.ListLabelsRequest;
039import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.ListLabelsResponse;
040import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.SetAuthsRequest;
041import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.VisibilityLabel;
042import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.VisibilityLabelsRequest;
043import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.VisibilityLabelsResponse;
044import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.VisibilityLabelsService;
045import org.apache.hadoop.hbase.util.ByteStringer;
046import org.apache.hadoop.hbase.util.Bytes;
047import org.apache.yetus.audience.InterfaceAudience;
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 n
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. nnnn * @deprecated Use
067   * {@link #addLabel(Connection,String)} instead.
068   */
069  @Deprecated
070  public static VisibilityLabelsResponse addLabel(Configuration conf, final String label)
071    throws Throwable {
072    try (Connection connection = ConnectionFactory.createConnection(conf)) {
073      return addLabels(connection, new String[] { label });
074    }
075  }
076
077  /**
078   * Utility method for adding label to the system. nnnn
079   */
080  public static VisibilityLabelsResponse addLabel(Connection connection, final String label)
081    throws Throwable {
082    return addLabels(connection, new String[] { label });
083  }
084
085  /**
086   * Utility method for adding labels to the system. nnnn * @deprecated Use
087   * {@link #addLabels(Connection,String[])} instead.
088   */
089  @Deprecated
090  public static VisibilityLabelsResponse addLabels(Configuration conf, final String[] labels)
091    throws Throwable {
092    try (Connection connection = ConnectionFactory.createConnection(conf)) {
093      return addLabels(connection, labels);
094    }
095  }
096
097  /**
098   * Utility method for adding labels to the system. nnnn
099   */
100  public static VisibilityLabelsResponse addLabels(Connection connection, final String[] labels)
101    throws Throwable {
102    try (Table table = connection.getTable(LABELS_TABLE_NAME)) {
103      Batch.Call<VisibilityLabelsService, VisibilityLabelsResponse> callable =
104        new Batch.Call<VisibilityLabelsService, VisibilityLabelsResponse>() {
105          ServerRpcController controller = new ServerRpcController();
106          CoprocessorRpcUtils.BlockingRpcCallback<VisibilityLabelsResponse> rpcCallback =
107            new CoprocessorRpcUtils.BlockingRpcCallback<>();
108
109          @Override
110          public VisibilityLabelsResponse call(VisibilityLabelsService service) throws IOException {
111            VisibilityLabelsRequest.Builder builder = VisibilityLabelsRequest.newBuilder();
112            for (String label : labels) {
113              if (label.length() > 0) {
114                VisibilityLabel.Builder newBuilder = VisibilityLabel.newBuilder();
115                newBuilder.setLabel(ByteStringer.wrap(Bytes.toBytes(label)));
116                builder.addVisLabel(newBuilder.build());
117              }
118            }
119            service.addLabels(controller, builder.build(), rpcCallback);
120            VisibilityLabelsResponse response = rpcCallback.get();
121            if (controller.failedOnException()) {
122              throw controller.getFailedOn();
123            }
124            return response;
125          }
126        };
127      Map<byte[], VisibilityLabelsResponse> result =
128        table.coprocessorService(VisibilityLabelsService.class, HConstants.EMPTY_BYTE_ARRAY,
129          HConstants.EMPTY_BYTE_ARRAY, callable);
130      return result.values().iterator().next(); // There will be exactly one region for labels
131      // table and so one entry in result Map.
132    }
133  }
134
135  /**
136   * Sets given labels globally authorized for the user. nnnnn * @deprecated Use
137   * {@link #setAuths(Connection,String[],String)} instead.
138   */
139  @Deprecated
140  public static VisibilityLabelsResponse setAuths(Configuration conf, final String[] auths,
141    final String user) throws Throwable {
142    try (Connection connection = ConnectionFactory.createConnection(conf)) {
143      return setOrClearAuths(connection, auths, user, true);
144    }
145  }
146
147  /**
148   * Sets given labels globally authorized for the user. nnnnn
149   */
150  public static VisibilityLabelsResponse setAuths(Connection connection, final String[] auths,
151    final String user) throws Throwable {
152    return setOrClearAuths(connection, auths, user, true);
153  }
154
155  /**
156   * Returns labels, the given user is globally authorized for. n * @deprecated Use
157   * {@link #getAuths(Connection,String)} instead.
158   */
159  @Deprecated
160  public static GetAuthsResponse getAuths(Configuration conf, final String user) throws Throwable {
161    try (Connection connection = ConnectionFactory.createConnection(conf)) {
162      return getAuths(connection, user);
163    }
164  }
165
166  /**
167   * Get the authorization for a given user
168   * @param connection the Connection instance to use
169   * @param user       the user
170   * @return labels the given user is globally authorized for
171   */
172  public static GetAuthsResponse getAuths(Connection connection, final String user)
173    throws Throwable {
174    try (Table table = connection.getTable(LABELS_TABLE_NAME)) {
175      Batch.Call<VisibilityLabelsService, GetAuthsResponse> callable =
176        new Batch.Call<VisibilityLabelsService, GetAuthsResponse>() {
177          ServerRpcController controller = new ServerRpcController();
178          CoprocessorRpcUtils.BlockingRpcCallback<GetAuthsResponse> rpcCallback =
179            new CoprocessorRpcUtils.BlockingRpcCallback<>();
180
181          @Override
182          public GetAuthsResponse call(VisibilityLabelsService service) throws IOException {
183            GetAuthsRequest.Builder getAuthReqBuilder = GetAuthsRequest.newBuilder();
184            getAuthReqBuilder.setUser(ByteStringer.wrap(Bytes.toBytes(user)));
185            service.getAuths(controller, getAuthReqBuilder.build(), rpcCallback);
186            GetAuthsResponse response = rpcCallback.get();
187            if (controller.failedOnException()) {
188              throw controller.getFailedOn();
189            }
190            return response;
191          }
192        };
193      Map<byte[], GetAuthsResponse> result = table.coprocessorService(VisibilityLabelsService.class,
194        HConstants.EMPTY_BYTE_ARRAY, HConstants.EMPTY_BYTE_ARRAY, callable);
195      return result.values().iterator().next(); // There will be exactly one region for labels
196      // table and so one entry in result Map.
197    }
198  }
199
200  /**
201   * Retrieve the list of visibility labels defined in the system.
202   * @param conf  the configuration to use
203   * @param regex The regular expression to filter which labels are returned.
204   * @return labels The list of visibility labels defined in the system.
205   * @deprecated Use {@link #listLabels(Connection,String)} instead.
206   */
207  @Deprecated
208  public static ListLabelsResponse listLabels(Configuration conf, final String regex)
209    throws Throwable {
210    try (Connection connection = ConnectionFactory.createConnection(conf)) {
211      return listLabels(connection, regex);
212    }
213  }
214
215  /**
216   * Retrieve the list of visibility labels defined in the system.
217   * @param connection The Connection instance to use.
218   * @param regex      The regular expression to filter which labels are returned.
219   * @return labels The list of visibility labels defined in the system. n
220   */
221  public static ListLabelsResponse listLabels(Connection connection, final String regex)
222    throws Throwable {
223    try (Table table = connection.getTable(LABELS_TABLE_NAME)) {
224      Batch.Call<VisibilityLabelsService, ListLabelsResponse> callable =
225        new Batch.Call<VisibilityLabelsService, ListLabelsResponse>() {
226          ServerRpcController controller = new ServerRpcController();
227          CoprocessorRpcUtils.BlockingRpcCallback<ListLabelsResponse> rpcCallback =
228            new CoprocessorRpcUtils.BlockingRpcCallback<>();
229
230          @Override
231          public ListLabelsResponse call(VisibilityLabelsService service) throws IOException {
232            ListLabelsRequest.Builder listAuthLabelsReqBuilder = ListLabelsRequest.newBuilder();
233            if (regex != null) {
234              // Compile the regex here to catch any regex exception earlier.
235              Pattern pattern = Pattern.compile(regex);
236              listAuthLabelsReqBuilder.setRegex(pattern.toString());
237            }
238            service.listLabels(controller, listAuthLabelsReqBuilder.build(), rpcCallback);
239            ListLabelsResponse response = rpcCallback.get();
240            if (controller.failedOnException()) {
241              throw controller.getFailedOn();
242            }
243            return response;
244          }
245        };
246      Map<byte[], ListLabelsResponse> result =
247        table.coprocessorService(VisibilityLabelsService.class, HConstants.EMPTY_BYTE_ARRAY,
248          HConstants.EMPTY_BYTE_ARRAY, callable);
249      return result.values().iterator().next(); // There will be exactly one region for labels
250      // table and so one entry in result Map.
251    }
252  }
253
254  /**
255   * Removes given labels from user's globally authorized list of labels. nnnnn * @deprecated Use
256   * {@link #clearAuths(Connection,String[],String)} instead.
257   */
258  @Deprecated
259  public static VisibilityLabelsResponse clearAuths(Configuration conf, final String[] auths,
260    final String user) throws Throwable {
261    try (Connection connection = ConnectionFactory.createConnection(conf)) {
262      return setOrClearAuths(connection, auths, user, false);
263    }
264  }
265
266  /**
267   * Removes given labels from user's globally authorized list of labels. nnnnn
268   */
269  public static VisibilityLabelsResponse clearAuths(Connection connection, final String[] auths,
270    final String user) throws Throwable {
271    return setOrClearAuths(connection, auths, user, false);
272  }
273
274  private static VisibilityLabelsResponse setOrClearAuths(Connection connection,
275    final String[] auths, final String user, final boolean setOrClear)
276    throws IOException, ServiceException, Throwable {
277
278    try (Table table = connection.getTable(LABELS_TABLE_NAME)) {
279      Batch.Call<VisibilityLabelsService, VisibilityLabelsResponse> callable =
280        new Batch.Call<VisibilityLabelsService, VisibilityLabelsResponse>() {
281          ServerRpcController controller = new ServerRpcController();
282          CoprocessorRpcUtils.BlockingRpcCallback<VisibilityLabelsResponse> rpcCallback =
283            new CoprocessorRpcUtils.BlockingRpcCallback<>();
284
285          @Override
286          public VisibilityLabelsResponse call(VisibilityLabelsService service) throws IOException {
287            SetAuthsRequest.Builder setAuthReqBuilder = SetAuthsRequest.newBuilder();
288            setAuthReqBuilder.setUser(ByteStringer.wrap(Bytes.toBytes(user)));
289            for (String auth : auths) {
290              if (auth.length() > 0) {
291                setAuthReqBuilder.addAuth(ByteString.copyFromUtf8(auth));
292              }
293            }
294            if (setOrClear) {
295              service.setAuths(controller, setAuthReqBuilder.build(), rpcCallback);
296            } else {
297              service.clearAuths(controller, setAuthReqBuilder.build(), rpcCallback);
298            }
299            VisibilityLabelsResponse response = rpcCallback.get();
300            if (controller.failedOnException()) {
301              throw controller.getFailedOn();
302            }
303            return response;
304          }
305        };
306      Map<byte[], VisibilityLabelsResponse> result =
307        table.coprocessorService(VisibilityLabelsService.class, HConstants.EMPTY_BYTE_ARRAY,
308          HConstants.EMPTY_BYTE_ARRAY, callable);
309      return result.values().iterator().next(); // There will be exactly one region for labels
310      // table and so one entry in result Map.
311    }
312  }
313}