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  
22  import org.apache.commons.logging.Log;
23  import org.apache.commons.logging.LogFactory;
24  import org.apache.hadoop.hbase.classification.InterfaceAudience;
25  import org.apache.hadoop.conf.Configuration;
26  import org.apache.hadoop.hbase.zookeeper.ZKUtil;
27  import org.apache.hadoop.hbase.zookeeper.ZooKeeperListener;
28  import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
29  import org.apache.zookeeper.KeeperException;
30  
31  /**
32   * A zk watcher that watches the labels table znode. This would create a znode
33   * /hbase/visibility_labels and will have a serialized form of a set of labels in the system.
34   */
35  @InterfaceAudience.Private
36  public class ZKVisibilityLabelWatcher extends ZooKeeperListener {
37  
38    private static final Log LOG = LogFactory.getLog(ZKVisibilityLabelWatcher.class);
39    private static final String VISIBILITY_LABEL_ZK_PATH = "zookeeper.znode.visibility.label.parent";
40    private static final String DEFAULT_VISIBILITY_LABEL_NODE = "visibility/labels";
41    private static final String VISIBILITY_USER_AUTHS_ZK_PATH = 
42        "zookeeper.znode.visibility.user.auths.parent";
43    private static final String DEFAULT_VISIBILITY_USER_AUTHS_NODE = "visibility/user_auths";
44  
45    private VisibilityLabelsCache labelsCache;
46    private String labelZnode;
47    private String userAuthsZnode;
48  
49    public ZKVisibilityLabelWatcher(ZooKeeperWatcher watcher, VisibilityLabelsCache labelsCache,
50        Configuration conf) {
51      super(watcher);
52      this.labelsCache = labelsCache;
53      String labelZnodeParent = conf.get(VISIBILITY_LABEL_ZK_PATH, DEFAULT_VISIBILITY_LABEL_NODE);
54      String userAuthsZnodeParent = conf.get(VISIBILITY_USER_AUTHS_ZK_PATH,
55          DEFAULT_VISIBILITY_USER_AUTHS_NODE);
56      this.labelZnode = ZKUtil.joinZNode(watcher.baseZNode, labelZnodeParent);
57      this.userAuthsZnode = ZKUtil.joinZNode(watcher.baseZNode, userAuthsZnodeParent);
58    }
59  
60    public void start() throws KeeperException {
61      watcher.registerListener(this);
62      ZKUtil.createWithParents(watcher, labelZnode);
63      ZKUtil.createWithParents(watcher, userAuthsZnode);
64      byte[] data = ZKUtil.getDataAndWatch(watcher, labelZnode);
65      if (data != null && data.length > 0) {
66        refreshVisibilityLabelsCache(data);
67      }
68      data = ZKUtil.getDataAndWatch(watcher, userAuthsZnode);
69      if (data != null && data.length > 0) {
70        refreshUserAuthsCache(data);
71      }
72    }
73  
74    private void refreshVisibilityLabelsCache(byte[] data) {
75      try {
76        this.labelsCache.refreshLabelsCache(data);
77      } catch (IOException ioe) {
78        LOG.error("Failed parsing data from labels table " + " from zk", ioe);
79      }
80    }
81  
82    private void refreshUserAuthsCache(byte[] data) {
83      try {
84        this.labelsCache.refreshUserAuthsCache(data);
85      } catch (IOException ioe) {
86        LOG.error("Failed parsing data from labels table " + " from zk", ioe);
87      }
88    }
89  
90    @Override
91    public void nodeCreated(String path) {
92      if (path.equals(labelZnode) || path.equals(userAuthsZnode)) {
93        try {
94          ZKUtil.watchAndCheckExists(watcher, path);
95        } catch (KeeperException ke) {
96          LOG.error("Error setting watcher on node " + path, ke);
97          // only option is to abort
98          watcher.abort("Zookeeper error obtaining label node children", ke);
99        }
100     }
101   }
102 
103   @Override
104   public void nodeDeleted(String path) {
105     // There is no case of visibility labels path to get deleted.
106   }
107 
108   @Override
109   public void nodeDataChanged(String path) {
110     if (path.equals(labelZnode) || path.equals(userAuthsZnode)) {
111       try {
112         watcher.sync(path);
113         byte[] data = ZKUtil.getDataAndWatch(watcher, path);
114         if (path.equals(labelZnode)) {
115           refreshVisibilityLabelsCache(data);
116         } else {
117           refreshUserAuthsCache(data);
118         }
119       } catch (KeeperException ke) {
120         LOG.error("Error reading data from zookeeper for node " + path, ke);
121         // only option is to abort
122         watcher.abort("Zookeeper error getting data for node " + path, ke);
123       }
124     }
125   }
126 
127   @Override
128   public void nodeChildrenChanged(String path) {
129     // We are not dealing with child nodes under the label znode or userauths znode.
130   }
131 
132   /**
133    * Write a labels mirror or user auths mirror into zookeeper
134    * 
135    * @param data
136    * @param labelsOrUserAuths true for writing labels and false for user auths.
137    */
138   public void writeToZookeeper(byte[] data, boolean labelsOrUserAuths) {
139     String znode = this.labelZnode;
140     if (!labelsOrUserAuths) {
141       znode = this.userAuthsZnode;
142     }
143     try {
144       ZKUtil.updateExistingNodeData(watcher, znode, data, -1);
145     } catch (KeeperException e) {
146       LOG.error("Failed writing to " + znode, e);
147       watcher.abort("Failed writing node " + znode + " to zookeeper", e);
148     }
149   }
150 }