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 java.io.IOException; 021import java.util.ArrayList; 022import java.util.Collections; 023import java.util.HashMap; 024import java.util.HashSet; 025import java.util.List; 026import java.util.Map; 027import java.util.Set; 028import java.util.concurrent.locks.ReentrantReadWriteLock; 029import org.apache.hadoop.conf.Configuration; 030import org.apache.hadoop.hbase.AuthUtil; 031import org.apache.hadoop.hbase.exceptions.DeserializationException; 032import org.apache.hadoop.hbase.util.Bytes; 033import org.apache.hadoop.hbase.zookeeper.ZKWatcher; 034import org.apache.yetus.audience.InterfaceAudience; 035import org.apache.zookeeper.KeeperException; 036import org.slf4j.Logger; 037import org.slf4j.LoggerFactory; 038 039import org.apache.hadoop.hbase.shaded.protobuf.generated.VisibilityLabelsProtos.MultiUserAuthorizations; 040import org.apache.hadoop.hbase.shaded.protobuf.generated.VisibilityLabelsProtos.UserAuthorizations; 041import org.apache.hadoop.hbase.shaded.protobuf.generated.VisibilityLabelsProtos.VisibilityLabel; 042 043/** 044 * Maintains the cache for visibility labels and also uses the zookeeper to update the labels in the 045 * system. The cache updation happens based on the data change event that happens on the zookeeper 046 * znode for labels table 047 */ 048@InterfaceAudience.Private 049public class VisibilityLabelsCache implements VisibilityLabelOrdinalProvider { 050 051 private static final Logger LOG = LoggerFactory.getLogger(VisibilityLabelsCache.class); 052 private static final List<String> EMPTY_LIST = Collections.emptyList(); 053 private static final Set<Integer> EMPTY_SET = Collections.emptySet(); 054 private static VisibilityLabelsCache instance; 055 056 private ZKVisibilityLabelWatcher zkVisibilityWatcher; 057 private Map<String, Integer> labels = new HashMap<>(); 058 private Map<Integer, String> ordinalVsLabels = new HashMap<>(); 059 private Map<String, Set<Integer>> userAuths = new HashMap<>(); 060 private Map<String, Set<Integer>> groupAuths = new HashMap<>(); 061 062 /** 063 * This covers the members labels, ordinalVsLabels and userAuths 064 */ 065 private ReentrantReadWriteLock lock = new ReentrantReadWriteLock(); 066 067 private VisibilityLabelsCache(ZKWatcher watcher, Configuration conf) throws IOException { 068 zkVisibilityWatcher = new ZKVisibilityLabelWatcher(watcher, this, conf); 069 try { 070 zkVisibilityWatcher.start(); 071 } catch (KeeperException ke) { 072 LOG.error("ZooKeeper initialization failed", ke); 073 throw new IOException(ke); 074 } 075 } 076 077 /** 078 * Creates the singleton instance, if not yet present, and returns the same. 079 * @return Singleton instance of VisibilityLabelsCache 080 */ 081 @edu.umd.cs.findbugs.annotations.SuppressWarnings(value = "MS_EXPOSE_REP", 082 justification = "singleton pattern") 083 public synchronized static VisibilityLabelsCache createAndGet(ZKWatcher watcher, 084 Configuration conf) throws IOException { 085 // VisibilityLabelService#init() for different regions (in same RS) passes same instance of 086 // watcher as all get the instance from RS. 087 // watcher != instance.zkVisibilityWatcher.getWatcher() - This check is needed only in UTs with 088 // RS restart. It will be same JVM in which RS restarts and instance will be not null. But the 089 // watcher associated with existing instance will be stale as the restarted RS will have new 090 // watcher with it. 091 if (instance == null || watcher != instance.zkVisibilityWatcher.getWatcher()) { 092 instance = new VisibilityLabelsCache(watcher, conf); 093 } 094 return instance; 095 } 096 097 /** 098 * @return Singleton instance of VisibilityLabelsCache when this is called before calling 099 * {@link #createAndGet(ZKWatcher, Configuration)} 100 */ 101 @edu.umd.cs.findbugs.annotations.SuppressWarnings(value = "MS_EXPOSE_REP", 102 justification = "singleton pattern") 103 public static VisibilityLabelsCache get() { 104 // By the time this method is called, the singleton instance of VisibilityLabelsCache should 105 // have been created. 106 if (instance == null) { 107 throw new IllegalStateException("VisibilityLabelsCache not yet instantiated"); 108 } 109 return instance; 110 } 111 112 public void refreshLabelsCache(byte[] data) throws IOException { 113 List<VisibilityLabel> visibilityLabels = null; 114 try { 115 visibilityLabels = VisibilityUtils.readLabelsFromZKData(data); 116 } catch (DeserializationException dse) { 117 throw new IOException(dse); 118 } 119 this.lock.writeLock().lock(); 120 try { 121 labels.clear(); 122 ordinalVsLabels.clear(); 123 for (VisibilityLabel visLabel : visibilityLabels) { 124 String label = Bytes.toString(visLabel.getLabel().toByteArray()); 125 labels.put(label, visLabel.getOrdinal()); 126 ordinalVsLabels.put(visLabel.getOrdinal(), label); 127 } 128 } finally { 129 this.lock.writeLock().unlock(); 130 } 131 } 132 133 public void refreshUserAuthsCache(byte[] data) throws IOException { 134 MultiUserAuthorizations multiUserAuths = null; 135 try { 136 multiUserAuths = VisibilityUtils.readUserAuthsFromZKData(data); 137 } catch (DeserializationException dse) { 138 throw new IOException(dse); 139 } 140 this.lock.writeLock().lock(); 141 try { 142 this.userAuths.clear(); 143 this.groupAuths.clear(); 144 for (UserAuthorizations userAuths : multiUserAuths.getUserAuthsList()) { 145 String user = Bytes.toString(userAuths.getUser().toByteArray()); 146 if (AuthUtil.isGroupPrincipal(user)) { 147 this.groupAuths.put(AuthUtil.getGroupName(user), new HashSet<>(userAuths.getAuthList())); 148 } else { 149 this.userAuths.put(user, new HashSet<>(userAuths.getAuthList())); 150 } 151 } 152 } finally { 153 this.lock.writeLock().unlock(); 154 } 155 } 156 157 /** 158 * @param label Not null label string 159 * @return The ordinal for the label. The ordinal starts from 1. Returns 0 when passed a non 160 * existing label. 161 */ 162 @Override 163 public int getLabelOrdinal(String label) { 164 Integer ordinal = null; 165 this.lock.readLock().lock(); 166 try { 167 ordinal = labels.get(label); 168 } finally { 169 this.lock.readLock().unlock(); 170 } 171 if (ordinal != null) { 172 return ordinal.intValue(); 173 } 174 // 0 denotes not available 175 return VisibilityConstants.NON_EXIST_LABEL_ORDINAL; 176 } 177 178 /** 179 * @param ordinal The ordinal of label which we are looking for. 180 * @return The label having the given ordinal. Returns <code>null</code> when no label exist in 181 * the system with given ordinal 182 */ 183 @Override 184 public String getLabel(int ordinal) { 185 this.lock.readLock().lock(); 186 try { 187 return this.ordinalVsLabels.get(ordinal); 188 } finally { 189 this.lock.readLock().unlock(); 190 } 191 } 192 193 /** Returns The total number of visibility labels. */ 194 public int getLabelsCount() { 195 this.lock.readLock().lock(); 196 try { 197 return this.labels.size(); 198 } finally { 199 this.lock.readLock().unlock(); 200 } 201 } 202 203 public List<String> getUserAuths(String user) { 204 this.lock.readLock().lock(); 205 try { 206 List<String> auths = EMPTY_LIST; 207 Set<Integer> authOrdinals = getUserAuthsAsOrdinals(user); 208 if (!authOrdinals.equals(EMPTY_SET)) { 209 auths = new ArrayList<>(authOrdinals.size()); 210 for (Integer authOrdinal : authOrdinals) { 211 auths.add(ordinalVsLabels.get(authOrdinal)); 212 } 213 } 214 return auths; 215 } finally { 216 this.lock.readLock().unlock(); 217 } 218 } 219 220 public List<String> getGroupAuths(String[] groups) { 221 this.lock.readLock().lock(); 222 try { 223 List<String> auths = EMPTY_LIST; 224 Set<Integer> authOrdinals = getGroupAuthsAsOrdinals(groups); 225 if (!authOrdinals.equals(EMPTY_SET)) { 226 auths = new ArrayList<>(authOrdinals.size()); 227 for (Integer authOrdinal : authOrdinals) { 228 auths.add(ordinalVsLabels.get(authOrdinal)); 229 } 230 } 231 return auths; 232 } finally { 233 this.lock.readLock().unlock(); 234 } 235 } 236 237 /** 238 * Returns the list of ordinals of labels associated with the user 239 * @param user Not null value. 240 * @return the list of ordinals 241 */ 242 public Set<Integer> getUserAuthsAsOrdinals(String user) { 243 this.lock.readLock().lock(); 244 try { 245 Set<Integer> auths = userAuths.get(user); 246 return (auths == null) ? EMPTY_SET : auths; 247 } finally { 248 this.lock.readLock().unlock(); 249 } 250 } 251 252 /** 253 * Returns the list of ordinals of labels associated with the groups 254 * @return the list of ordinals 255 */ 256 public Set<Integer> getGroupAuthsAsOrdinals(String[] groups) { 257 this.lock.readLock().lock(); 258 try { 259 Set<Integer> authOrdinals = new HashSet<>(); 260 if (groups != null && groups.length > 0) { 261 Set<Integer> groupAuthOrdinals = null; 262 for (String group : groups) { 263 groupAuthOrdinals = groupAuths.get(group); 264 if (groupAuthOrdinals != null && !groupAuthOrdinals.isEmpty()) { 265 authOrdinals.addAll(groupAuthOrdinals); 266 } 267 } 268 } 269 return (authOrdinals.isEmpty()) ? EMPTY_SET : authOrdinals; 270 } finally { 271 this.lock.readLock().unlock(); 272 } 273 } 274 275 public void writeToZookeeper(byte[] data, boolean labelsOrUserAuths) throws IOException { 276 // Update local state, then send it to zookeeper 277 if (labelsOrUserAuths) { 278 // True for labels 279 this.refreshLabelsCache(data); 280 } else { 281 // False for user auths 282 this.refreshUserAuthsCache(data); 283 } 284 this.zkVisibilityWatcher.writeToZookeeper(data, labelsOrUserAuths); 285 } 286}