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