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}