1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
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
33
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
98 watcher.abort("Zookeeper error obtaining label node children", ke);
99 }
100 }
101 }
102
103 @Override
104 public void nodeDeleted(String path) {
105
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
122 watcher.abort("Zookeeper error getting data for node " + path, ke);
123 }
124 }
125 }
126
127 @Override
128 public void nodeChildrenChanged(String path) {
129
130 }
131
132
133
134
135
136
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 }