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.HashMap;
23  import java.util.HashSet;
24  import java.util.List;
25  import java.util.Map;
26  import java.util.Set;
27  import java.util.concurrent.locks.ReentrantReadWriteLock;
28  
29  import org.apache.commons.logging.Log;
30  import org.apache.commons.logging.LogFactory;
31  import org.apache.hadoop.classification.InterfaceAudience;
32  import org.apache.hadoop.conf.Configuration;
33  import org.apache.hadoop.hbase.exceptions.DeserializationException;
34  import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.MultiUserAuthorizations;
35  import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.UserAuthorizations;
36  import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.VisibilityLabel;
37  import org.apache.hadoop.hbase.util.Bytes;
38  import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
39  import org.apache.zookeeper.KeeperException;
40  
41  /**
42   * Maintains the cache for visibility labels and also uses the zookeeper to update the labels in the
43   * system. The cache updation happens based on the data change event that happens on the zookeeper
44   * znode for labels table
45   */
46  @InterfaceAudience.Private
47  public class VisibilityLabelsManager {
48  
49    private static final Log LOG = LogFactory.getLog(VisibilityLabelsManager.class);
50    private static final List<String> EMPTY_LIST = new ArrayList<String>(0);
51    private static VisibilityLabelsManager instance;
52  
53    private ZKVisibilityLabelWatcher zkVisibilityWatcher;
54    private Map<String, Integer> labels = new HashMap<String, Integer>();
55    private Map<Integer, String> ordinalVsLabels = new HashMap<Integer, String>();
56    private Map<String, Set<Integer>> userAuths = new HashMap<String, Set<Integer>>();
57    private ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
58  
59    private VisibilityLabelsManager(ZooKeeperWatcher watcher, Configuration conf) throws IOException {
60      zkVisibilityWatcher = new ZKVisibilityLabelWatcher(watcher, this, conf);
61      try {
62        zkVisibilityWatcher.start();
63      } catch (KeeperException ke) {
64        LOG.error("ZooKeeper initialization failed", ke);
65        throw new IOException(ke);
66      }
67    }
68  
69    public synchronized static VisibilityLabelsManager get(ZooKeeperWatcher watcher,
70        Configuration conf) throws IOException {
71      if (instance == null) {
72        instance = new VisibilityLabelsManager(watcher, conf);
73      }
74      return instance;
75    }
76  
77    public static VisibilityLabelsManager get() {
78      return instance;
79    }
80  
81    public void refreshLabelsCache(byte[] data) throws IOException {
82      List<VisibilityLabel> visibilityLabels = null;
83      try {
84        visibilityLabels = VisibilityUtils.readLabelsFromZKData(data);
85      } catch (DeserializationException dse) {
86        throw new IOException(dse);
87      }
88      this.lock.writeLock().lock();
89      try {
90        for (VisibilityLabel visLabel : visibilityLabels) {
91          String label = Bytes.toString(visLabel.getLabel().toByteArray());
92          labels.put(label, visLabel.getOrdinal());
93          ordinalVsLabels.put(visLabel.getOrdinal(), label);
94        }
95      } finally {
96        this.lock.writeLock().unlock();
97      }
98    }
99  
100   public void refreshUserAuthsCache(byte[] data) throws IOException {
101     MultiUserAuthorizations multiUserAuths = null;
102     try {
103       multiUserAuths = VisibilityUtils.readUserAuthsFromZKData(data);
104     } catch (DeserializationException dse) {
105       throw new IOException(dse);
106     }
107     this.lock.writeLock().lock();
108     try {
109       for (UserAuthorizations userAuths : multiUserAuths.getUserAuthsList()) {
110         String user = Bytes.toString(userAuths.getUser().toByteArray());
111         this.userAuths.put(user, new HashSet<Integer>(userAuths.getAuthList()));
112       }
113     } finally {
114       this.lock.writeLock().unlock();
115     }
116   }
117 
118   /**
119    * @param label
120    * @return The ordinal for the label. The ordinal starts from 1. Returns 0 when the passed a non
121    *         existing label.
122    */
123   public int getLabelOrdinal(String label) {
124     Integer ordinal = null;
125     this.lock.readLock().lock();
126     try {
127       ordinal = labels.get(label);
128     } finally {
129       this.lock.readLock().unlock();
130     }
131     if (ordinal != null) {
132       return ordinal.intValue();
133     }
134     // 0 denotes not available
135     return 0;
136   }
137 
138   public String getLabel(int ordinal) {
139     this.lock.readLock().lock();
140     try {
141       return this.ordinalVsLabels.get(ordinal);
142     } finally {
143       this.lock.readLock().unlock();
144     }
145   }
146 
147   /**
148    * @return The total number of visibility labels.
149    */
150   public int getLabelsCount(){
151     return this.labels.size();
152   }
153 
154   /**
155    * @param user
156    * @return The labels that the given user is authorized for.
157    */
158   public List<String> getAuths(String user) {
159     List<String> auths = EMPTY_LIST;
160     this.lock.readLock().lock();
161     try {
162       Set<Integer> authOrdinals = userAuths.get(user);
163       if (authOrdinals != null) {
164         auths = new ArrayList<String>(authOrdinals.size());
165         for (Integer authOrdinal : authOrdinals) {
166           auths.add(ordinalVsLabels.get(authOrdinal));
167         }
168       }
169     } finally {
170       this.lock.readLock().unlock();
171     }
172     return auths;
173   }
174 
175   /**
176    * Writes the labels data to zookeeper node.
177    * @param data
178    * @param labelsOrUserAuths true for writing labels and false for user auths.
179    */
180   public void writeToZookeeper(byte[] data, boolean labelsOrUserAuths) {
181     this.zkVisibilityWatcher.writeToZookeeper(data, labelsOrUserAuths);
182   }
183 }