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}