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. nn * @return 078 * Singleton instance of VisibilityLabelsCache n 079 */ 080 public synchronized static VisibilityLabelsCache createAndGet(ZKWatcher watcher, 081 Configuration conf) throws IOException { 082 // VisibilityLabelService#init() for different regions (in same RS) passes same instance of 083 // watcher as all get the instance from RS. 084 // watcher != instance.zkVisibilityWatcher.getWatcher() - This check is needed only in UTs with 085 // RS restart. It will be same JVM in which RS restarts and instance will be not null. But the 086 // watcher associated with existing instance will be stale as the restarted RS will have new 087 // watcher with it. 088 if (instance == null || watcher != instance.zkVisibilityWatcher.getWatcher()) { 089 instance = new VisibilityLabelsCache(watcher, conf); 090 } 091 return instance; 092 } 093 094 /** 095 * @return Singleton instance of VisibilityLabelsCache n * when this is called before calling 096 * {@link #createAndGet(ZKWatcher, Configuration)} 097 */ 098 public static VisibilityLabelsCache get() { 099 // By the time this method is called, the singleton instance of VisibilityLabelsCache should 100 // have been created. 101 if (instance == null) { 102 throw new IllegalStateException("VisibilityLabelsCache not yet instantiated"); 103 } 104 return instance; 105 } 106 107 public void refreshLabelsCache(byte[] data) throws IOException { 108 List<VisibilityLabel> visibilityLabels = null; 109 try { 110 visibilityLabels = VisibilityUtils.readLabelsFromZKData(data); 111 } catch (DeserializationException dse) { 112 throw new IOException(dse); 113 } 114 this.lock.writeLock().lock(); 115 try { 116 labels.clear(); 117 ordinalVsLabels.clear(); 118 for (VisibilityLabel visLabel : visibilityLabels) { 119 String label = Bytes.toString(visLabel.getLabel().toByteArray()); 120 labels.put(label, visLabel.getOrdinal()); 121 ordinalVsLabels.put(visLabel.getOrdinal(), label); 122 } 123 } finally { 124 this.lock.writeLock().unlock(); 125 } 126 } 127 128 public void refreshUserAuthsCache(byte[] data) throws IOException { 129 MultiUserAuthorizations multiUserAuths = null; 130 try { 131 multiUserAuths = VisibilityUtils.readUserAuthsFromZKData(data); 132 } catch (DeserializationException dse) { 133 throw new IOException(dse); 134 } 135 this.lock.writeLock().lock(); 136 try { 137 this.userAuths.clear(); 138 this.groupAuths.clear(); 139 for (UserAuthorizations userAuths : multiUserAuths.getUserAuthsList()) { 140 String user = Bytes.toString(userAuths.getUser().toByteArray()); 141 if (AuthUtil.isGroupPrincipal(user)) { 142 this.groupAuths.put(AuthUtil.getGroupName(user), new HashSet<>(userAuths.getAuthList())); 143 } else { 144 this.userAuths.put(user, new HashSet<>(userAuths.getAuthList())); 145 } 146 } 147 } finally { 148 this.lock.writeLock().unlock(); 149 } 150 } 151 152 /** 153 * @param label Not null label string 154 * @return The ordinal for the label. The ordinal starts from 1. Returns 0 when passed a non 155 * existing label. 156 */ 157 @Override 158 public int getLabelOrdinal(String label) { 159 Integer ordinal = null; 160 this.lock.readLock().lock(); 161 try { 162 ordinal = labels.get(label); 163 } finally { 164 this.lock.readLock().unlock(); 165 } 166 if (ordinal != null) { 167 return ordinal.intValue(); 168 } 169 // 0 denotes not available 170 return VisibilityConstants.NON_EXIST_LABEL_ORDINAL; 171 } 172 173 /** 174 * @param ordinal The ordinal of label which we are looking for. 175 * @return The label having the given ordinal. Returns <code>null</code> when no label exist in 176 * the system with given ordinal 177 */ 178 @Override 179 public String getLabel(int ordinal) { 180 this.lock.readLock().lock(); 181 try { 182 return this.ordinalVsLabels.get(ordinal); 183 } finally { 184 this.lock.readLock().unlock(); 185 } 186 } 187 188 /** Returns The total number of visibility labels. */ 189 public int getLabelsCount() { 190 this.lock.readLock().lock(); 191 try { 192 return this.labels.size(); 193 } finally { 194 this.lock.readLock().unlock(); 195 } 196 } 197 198 public List<String> getUserAuths(String user) { 199 this.lock.readLock().lock(); 200 try { 201 List<String> auths = EMPTY_LIST; 202 Set<Integer> authOrdinals = getUserAuthsAsOrdinals(user); 203 if (!authOrdinals.equals(EMPTY_SET)) { 204 auths = new ArrayList<>(authOrdinals.size()); 205 for (Integer authOrdinal : authOrdinals) { 206 auths.add(ordinalVsLabels.get(authOrdinal)); 207 } 208 } 209 return auths; 210 } finally { 211 this.lock.readLock().unlock(); 212 } 213 } 214 215 public List<String> getGroupAuths(String[] groups) { 216 this.lock.readLock().lock(); 217 try { 218 List<String> auths = EMPTY_LIST; 219 Set<Integer> authOrdinals = getGroupAuthsAsOrdinals(groups); 220 if (!authOrdinals.equals(EMPTY_SET)) { 221 auths = new ArrayList<>(authOrdinals.size()); 222 for (Integer authOrdinal : authOrdinals) { 223 auths.add(ordinalVsLabels.get(authOrdinal)); 224 } 225 } 226 return auths; 227 } finally { 228 this.lock.readLock().unlock(); 229 } 230 } 231 232 /** 233 * Returns the list of ordinals of labels associated with the user 234 * @param user Not null value. 235 * @return the list of ordinals 236 */ 237 public Set<Integer> getUserAuthsAsOrdinals(String user) { 238 this.lock.readLock().lock(); 239 try { 240 Set<Integer> auths = userAuths.get(user); 241 return (auths == null) ? EMPTY_SET : auths; 242 } finally { 243 this.lock.readLock().unlock(); 244 } 245 } 246 247 /** 248 * Returns the list of ordinals of labels associated with the groups n * @return the list of 249 * ordinals 250 */ 251 public Set<Integer> getGroupAuthsAsOrdinals(String[] groups) { 252 this.lock.readLock().lock(); 253 try { 254 Set<Integer> authOrdinals = new HashSet<>(); 255 if (groups != null && groups.length > 0) { 256 Set<Integer> groupAuthOrdinals = null; 257 for (String group : groups) { 258 groupAuthOrdinals = groupAuths.get(group); 259 if (groupAuthOrdinals != null && !groupAuthOrdinals.isEmpty()) { 260 authOrdinals.addAll(groupAuthOrdinals); 261 } 262 } 263 } 264 return (authOrdinals.isEmpty()) ? EMPTY_SET : authOrdinals; 265 } finally { 266 this.lock.readLock().unlock(); 267 } 268 } 269 270 public void writeToZookeeper(byte[] data, boolean labelsOrUserAuths) throws IOException { 271 // Update local state, then send it to zookeeper 272 if (labelsOrUserAuths) { 273 // True for labels 274 this.refreshLabelsCache(data); 275 } else { 276 // False for user auths 277 this.refreshUserAuthsCache(data); 278 } 279 this.zkVisibilityWatcher.writeToZookeeper(data, labelsOrUserAuths); 280 } 281}