View Javadoc

1   /*
2    * Licensed to the Apache Software Foundation (ASF) under one
3    * or more contributor license agreements.  See the NOTICE file
4    * distributed with this work for additional information
5    * regarding copyright ownership.  The ASF licenses this file
6    * to you under the Apache License, Version 2.0 (the
7    * "License"); you may not use this file except in compliance
8    * with the License.  You may obtain a copy of the License at
9    *
10   *     http://www.apache.org/licenses/LICENSE-2.0
11   *
12   * Unless required by applicable law or agreed to in writing, software
13   * distributed under the License is distributed on an "AS IS" BASIS,
14   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15   * See the License for the specific language governing permissions and
16   * limitations under the License.
17   */
18  package org.apache.hadoop.hbase.security.visibility;
19  
20  import java.io.IOException;
21  import java.util.ArrayList;
22  import java.util.Collections;
23  import java.util.HashMap;
24  import java.util.HashSet;
25  import java.util.List;
26  import java.util.Map;
27  import java.util.Set;
28  import java.util.concurrent.locks.ReentrantReadWriteLock;
29  
30  import org.apache.commons.logging.Log;
31  import org.apache.commons.logging.LogFactory;
32  import org.apache.hadoop.conf.Configuration;
33  import org.apache.hadoop.hbase.AuthUtil;
34  import org.apache.hadoop.hbase.classification.InterfaceAudience;
35  import org.apache.hadoop.hbase.exceptions.DeserializationException;
36  import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.MultiUserAuthorizations;
37  import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.UserAuthorizations;
38  import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.VisibilityLabel;
39  import org.apache.hadoop.hbase.util.Bytes;
40  import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
41  import org.apache.zookeeper.KeeperException;
42  
43  /**
44   * Maintains the cache for visibility labels and also uses the zookeeper to update the labels in the
45   * system. The cache updation happens based on the data change event that happens on the zookeeper
46   * znode for labels table
47   */
48  @InterfaceAudience.Private
49  public class VisibilityLabelsCache implements VisibilityLabelOrdinalProvider {
50  
51    private static final Log LOG = LogFactory.getLog(VisibilityLabelsCache.class);
52    private static final List<String> EMPTY_LIST = Collections.emptyList();
53    private static final Set<Integer> EMPTY_SET = Collections.emptySet();
54    private static VisibilityLabelsCache instance;
55  
56    private ZKVisibilityLabelWatcher zkVisibilityWatcher;
57    private Map<String, Integer> labels = new HashMap<String, Integer>();
58    private Map<Integer, String> ordinalVsLabels = new HashMap<Integer, String>();
59    private Map<String, Set<Integer>> userAuths = new HashMap<String, Set<Integer>>();
60    private Map<String, Set<Integer>> groupAuths = new HashMap<String, Set<Integer>>();
61  
62    /**
63     * This covers the members labels, ordinalVsLabels and userAuths
64     */
65    private ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
66  
67    private VisibilityLabelsCache(ZooKeeperWatcher watcher, Configuration conf) throws IOException {
68      zkVisibilityWatcher = new ZKVisibilityLabelWatcher(watcher, this, conf);
69      try {
70        zkVisibilityWatcher.start();
71      } catch (KeeperException ke) {
72        LOG.error("ZooKeeper initialization failed", ke);
73        throw new IOException(ke);
74      }
75    }
76  
77    /**
78     * Creates the singleton instance, if not yet present, and returns the same.
79     * @param watcher
80     * @param conf
81     * @return Singleton instance of VisibilityLabelsCache
82     * @throws IOException
83     */
84    public synchronized static VisibilityLabelsCache createAndGet(ZooKeeperWatcher watcher,
85        Configuration conf) throws IOException {
86      // VisibilityLabelService#init() for different regions (in same RS) passes same instance of
87      // watcher as all get the instance from RS.
88      // watcher != instance.zkVisibilityWatcher.getWatcher() - This check is needed only in UTs with
89      // RS restart. It will be same JVM in which RS restarts and instance will be not null. But the
90      // watcher associated with existing instance will be stale as the restarted RS will have new
91      // watcher with it.
92      if (instance == null || watcher != instance.zkVisibilityWatcher.getWatcher()) {
93        instance = new VisibilityLabelsCache(watcher, conf);
94      }
95      return instance;
96    }
97  
98    /**
99     * @return Singleton instance of VisibilityLabelsCache
100    * @throws IllegalStateException
101    *           when this is called before calling
102    *           {@link #createAndGet(ZooKeeperWatcher, Configuration)}
103    */
104   public static VisibilityLabelsCache get() {
105     // By the time this method is called, the singleton instance of VisibilityLabelsCache should
106     // have been created.
107     if (instance == null) {
108       throw new IllegalStateException("VisibilityLabelsCache not yet instantiated");
109     }
110     return instance;
111   }
112 
113   public void refreshLabelsCache(byte[] data) throws IOException {
114     List<VisibilityLabel> visibilityLabels = null;
115     try {
116       visibilityLabels = VisibilityUtils.readLabelsFromZKData(data);
117     } catch (DeserializationException dse) {
118       throw new IOException(dse);
119     }
120     this.lock.writeLock().lock();
121     try {
122       labels.clear();
123       ordinalVsLabels.clear();
124       for (VisibilityLabel visLabel : visibilityLabels) {
125         String label = Bytes.toString(visLabel.getLabel().toByteArray());
126         labels.put(label, visLabel.getOrdinal());
127         ordinalVsLabels.put(visLabel.getOrdinal(), label);
128       }
129     } finally {
130       this.lock.writeLock().unlock();
131     }
132   }
133 
134   public void refreshUserAuthsCache(byte[] data) throws IOException {
135     MultiUserAuthorizations multiUserAuths = null;
136     try {
137       multiUserAuths = VisibilityUtils.readUserAuthsFromZKData(data);
138     } catch (DeserializationException dse) {
139       throw new IOException(dse);
140     }
141     this.lock.writeLock().lock();
142     try {
143       this.userAuths.clear();
144       this.groupAuths.clear();
145       for (UserAuthorizations userAuths : multiUserAuths.getUserAuthsList()) {
146         String user = Bytes.toString(userAuths.getUser().toByteArray());
147         if (AuthUtil.isGroupPrincipal(user)) {
148           this.groupAuths.put(AuthUtil.getGroupName(user),
149             new HashSet<Integer>(userAuths.getAuthList()));
150         } else {
151           this.userAuths.put(user, new HashSet<Integer>(userAuths.getAuthList()));
152         }
153       }
154     } finally {
155       this.lock.writeLock().unlock();
156     }
157   }
158 
159   /**
160    * @param label Not null label string
161    * @return The ordinal for the label. The ordinal starts from 1. Returns 0 when passed a non
162    *         existing label.
163    */
164   @Override
165   public int getLabelOrdinal(String label) {
166     Integer ordinal = null;
167     this.lock.readLock().lock();
168     try {
169       ordinal = labels.get(label);
170     } finally {
171       this.lock.readLock().unlock();
172     }
173     if (ordinal != null) {
174       return ordinal.intValue();
175     }
176     // 0 denotes not available
177     return VisibilityConstants.NON_EXIST_LABEL_ORDINAL;
178   }
179 
180   /**
181    * @param ordinal The ordinal of label which we are looking for.
182    * @return The label having the given ordinal. Returns <code>null</code> when no label exist in
183    *         the system with given ordinal
184    */
185   @Override
186   public String getLabel(int ordinal) {
187     this.lock.readLock().lock();
188     try {
189       return this.ordinalVsLabels.get(ordinal);
190     } finally {
191       this.lock.readLock().unlock();
192     }
193   }
194 
195   /**
196    * @return The total number of visibility labels.
197    */
198   public int getLabelsCount() {
199     this.lock.readLock().lock();
200     try {
201       return this.labels.size();
202     } finally {
203       this.lock.readLock().unlock();
204     }
205   }
206 
207   public List<String> getUserAuths(String user) {
208     this.lock.readLock().lock();
209     try {
210       List<String> auths = EMPTY_LIST;
211       Set<Integer> authOrdinals = getUserAuthsAsOrdinals(user);
212       if (!authOrdinals.equals(EMPTY_SET)) {
213         auths = new ArrayList<String>(authOrdinals.size());
214         for (Integer authOrdinal : authOrdinals) {
215           auths.add(ordinalVsLabels.get(authOrdinal));
216         }
217       }
218       return auths;
219     } finally {
220       this.lock.readLock().unlock();
221     }
222   }
223 
224   public List<String> getGroupAuths(String[] groups) {
225     this.lock.readLock().lock();
226     try {
227       List<String> auths = EMPTY_LIST;
228       Set<Integer> authOrdinals = getGroupAuthsAsOrdinals(groups);
229       if (!authOrdinals.equals(EMPTY_SET)) {
230         auths = new ArrayList<String>(authOrdinals.size());
231         for (Integer authOrdinal : authOrdinals) {
232           auths.add(ordinalVsLabels.get(authOrdinal));
233         }
234       }
235       return auths;
236     } finally {
237       this.lock.readLock().unlock();
238     }
239   }
240 
241   /**
242    * Returns the list of ordinals of labels associated with the user
243    *
244    * @param user Not null value.
245    * @return the list of ordinals
246    */
247   public Set<Integer> getUserAuthsAsOrdinals(String user) {
248     this.lock.readLock().lock();
249     try {
250       Set<Integer> auths = userAuths.get(user);
251       return (auths == null) ? EMPTY_SET : auths;
252     } finally {
253       this.lock.readLock().unlock();
254     }
255   }
256 
257   /**
258    * Returns the list of ordinals of labels associated with the groups
259    *
260    * @param groups
261    * @return the list of ordinals
262    */
263   public Set<Integer> getGroupAuthsAsOrdinals(String[] groups) {
264     this.lock.readLock().lock();
265     try {
266       Set<Integer> authOrdinals = new HashSet<Integer>();
267       if (groups != null && groups.length > 0) {
268         Set<Integer> groupAuthOrdinals = null;
269         for (String group : groups) {
270           groupAuthOrdinals = groupAuths.get(group);
271           if (groupAuthOrdinals != null && !groupAuthOrdinals.isEmpty()) {
272             authOrdinals.addAll(groupAuthOrdinals);
273           }
274         }
275       }
276       return (authOrdinals.isEmpty()) ? EMPTY_SET : authOrdinals;
277     } finally {
278       this.lock.readLock().unlock();
279     }
280   }
281 
282   public void writeToZookeeper(byte[] data, boolean labelsOrUserAuths) throws IOException {
283     // Update local state, then send it to zookeeper
284     if (labelsOrUserAuths) {
285       // True for labels
286       this.refreshLabelsCache(data);
287     } else {
288       // False for user auths
289       this.refreshUserAuthsCache(data);
290     }
291     this.zkVisibilityWatcher.writeToZookeeper(data, labelsOrUserAuths);
292   }
293 }