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}