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 */
018
019package org.apache.hadoop.hbase.test;
020
021import static org.junit.Assert.assertEquals;
022import static org.junit.Assert.assertTrue;
023import static org.junit.Assert.fail;
024
025import java.io.IOException;
026import java.util.List;
027
028import org.apache.hadoop.conf.Configuration;
029import org.apache.hadoop.fs.FileSystem;
030import org.apache.hadoop.fs.Path;
031import org.apache.hadoop.fs.permission.FsPermission;
032import org.apache.hadoop.hbase.HBaseConfiguration;
033import org.apache.hadoop.hbase.IntegrationTestingUtility;
034import org.apache.hadoop.hbase.security.SecurityConstants;
035import org.apache.hadoop.hbase.testclassification.IntegrationTests;
036import org.apache.hadoop.hbase.util.AbstractHBaseTool;
037import org.apache.hadoop.hbase.util.FSUtils;
038import org.apache.hadoop.hbase.zookeeper.RecoverableZooKeeper;
039import org.apache.hadoop.hbase.zookeeper.ZKUtil;
040import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
041import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
042import org.apache.hadoop.util.ToolRunner;
043import org.apache.zookeeper.KeeperException;
044import org.apache.zookeeper.KeeperException.Code;
045import org.apache.zookeeper.KeeperException.NoNodeException;
046import org.apache.zookeeper.ZooDefs.Ids;
047import org.apache.zookeeper.ZooDefs.Perms;
048import org.apache.zookeeper.data.ACL;
049import org.apache.zookeeper.data.Id;
050import org.apache.zookeeper.data.Stat;
051import org.junit.experimental.categories.Category;
052import org.slf4j.Logger;
053import org.slf4j.LoggerFactory;
054
055import org.apache.hbase.thirdparty.org.apache.commons.cli.CommandLine;
056
057/**
058 * An integration test which checks that the znodes in zookeeper and data in the FileSystem
059 * are protected for secure HBase deployments.
060 * This test is intended to be run on clusters with kerberos authorization for HBase and ZooKeeper.
061 *
062 * If hbase.security.authentication is not set to kerberos, the test does not run unless -f is
063 * specified which bypasses the check. It is recommended to always run with -f on secure clusters
064 * so that the test checks the actual end result, not the configuration.
065 *
066 * The test should be run as hbase user with kinit / TGT cached since it accesses HDFS.
067 * <p>
068 * Example usage:
069 *   hbase org.apache.hadoop.hbase.test.IntegrationTestZnodeACLs -h
070 */
071@Category(IntegrationTests.class)
072public class IntegrationTestZKAndFSPermissions extends AbstractHBaseTool {
073  private static final Logger LOG =
074      LoggerFactory.getLogger(IntegrationTestZKAndFSPermissions.class);
075
076  private String superUser;
077  private String masterPrincipal;
078  private boolean isForce;
079  private String fsPerms;
080  private boolean skipFSCheck;
081  private boolean skipZKCheck;
082
083  public static final String FORCE_CHECK_ARG = "f";
084  public static final String PRINCIPAL_ARG = "p";
085  public static final String SUPERUSER_ARG = "s";
086  public static final String FS_PERMS = "fs_perms";
087  public static final String SKIP_CHECK_FS = "skip_fs_check";
088  public static final String SKIP_CHECK_ZK = "skip_zk_check";
089
090  @Override
091  public void setConf(Configuration conf) {
092    super.setConf(conf);
093  }
094
095  @Override
096  protected void addOptions() {
097    addOptNoArg(FORCE_CHECK_ARG, "Whether to skip configuration lookup and assume a secure setup");
098    addOptWithArg(PRINCIPAL_ARG, "The principal for zk authorization");
099    addOptWithArg(SUPERUSER_ARG, "The principal for super user");
100    addOptWithArg(FS_PERMS,      "FS permissions, ex. 700, 750, etc. Defaults to 700");
101    addOptNoArg(SKIP_CHECK_FS, "Whether to skip checking FS permissions");
102    addOptNoArg(SKIP_CHECK_ZK,   "Whether to skip checking ZK permissions");
103  }
104
105  @Override
106  protected void processOptions(CommandLine cmd) {
107    isForce = cmd.hasOption(FORCE_CHECK_ARG);
108    masterPrincipal = getShortUserName(conf.get(SecurityConstants.MASTER_KRB_PRINCIPAL));
109    superUser = cmd.getOptionValue(SUPERUSER_ARG, conf.get("hbase.superuser"));
110    masterPrincipal = cmd.getOptionValue(PRINCIPAL_ARG, masterPrincipal);
111    fsPerms = cmd.getOptionValue(FS_PERMS, "700");
112    skipFSCheck = cmd.hasOption(SKIP_CHECK_FS);
113    skipZKCheck = cmd.hasOption(SKIP_CHECK_ZK);
114  }
115
116  private String getShortUserName(String principal) {
117    for (int i = 0; i < principal.length(); i++) {
118      if (principal.charAt(i) == '/' || principal.charAt(i) == '@') {
119        return principal.substring(0, i);
120      }
121    }
122    return principal;
123  }
124
125  @Override
126  protected int doWork() throws Exception {
127    if (!isForce) {
128      if (!"kerberos".equalsIgnoreCase(conf.get("hbase.security.authentication"))) {
129        LOG.warn("hbase.security.authentication is not kerberos, and -f is not supplied. Skip "
130            + "running the test");
131        return 0;
132      }
133    }
134
135    if (!skipZKCheck) {
136      testZNodeACLs();
137    } if (!skipFSCheck) {
138      testFSPerms();
139    }
140    return 0;
141  }
142
143  private void testZNodeACLs() throws IOException, KeeperException, InterruptedException {
144
145    ZKWatcher watcher = new ZKWatcher(conf, "IntegrationTestZnodeACLs", null);
146    RecoverableZooKeeper zk = ZKUtil.connect(this.conf, watcher);
147
148    String baseZNode = watcher.getZNodePaths().baseZNode;
149
150    LOG.info("");
151    LOG.info("***********************************************************************************");
152    LOG.info("Checking ZK permissions, root znode: " + baseZNode);
153    LOG.info("***********************************************************************************");
154    LOG.info("");
155
156    checkZnodePermsRecursive(watcher, zk, baseZNode);
157
158    LOG.info("Checking ZK permissions: SUCCESS");
159  }
160
161  private void checkZnodePermsRecursive(ZKWatcher watcher,
162      RecoverableZooKeeper zk, String znode) throws KeeperException, InterruptedException {
163
164    boolean expectedWorldReadable = watcher.getZNodePaths().isClientReadable(znode);
165
166    assertZnodePerms(zk, znode, expectedWorldReadable);
167
168    try {
169      List<String> children = zk.getChildren(znode, false);
170
171      for (String child : children) {
172        checkZnodePermsRecursive(watcher, zk, ZNodePaths.joinZNode(znode, child));
173      }
174    } catch (KeeperException ke) {
175      // if we are not authenticated for listChildren, it is fine.
176      if (ke.code() != Code.NOAUTH && ke.code() != Code.NONODE) {
177        throw ke;
178      }
179    }
180  }
181
182  private void assertZnodePerms(RecoverableZooKeeper zk, String znode,
183      boolean expectedWorldReadable) throws KeeperException, InterruptedException {
184    Stat stat = new Stat();
185    List<ACL> acls;
186    try {
187      acls = zk.getZooKeeper().getACL(znode, stat);
188    } catch (NoNodeException ex) {
189      LOG.debug("Caught exception for missing znode", ex);
190      // the znode is deleted. Probably it was a temporary znode (like RIT).
191      return;
192    }
193    String[] superUsers = superUser == null ? null : superUser.split(",");
194
195    LOG.info("Checking ACLs for znode znode:" + znode + " acls:" + acls);
196
197    for (ACL acl : acls) {
198      int perms = acl.getPerms();
199      Id id = acl.getId();
200      // We should only set at most 3 possible ACL for 3 Ids. One for everyone, one for superuser
201      // and one for the hbase user
202      if (Ids.ANYONE_ID_UNSAFE.equals(id)) {
203        // everyone should be set only if we are expecting this znode to be world readable
204        assertTrue(expectedWorldReadable);
205        // assert that anyone can only read
206        assertEquals(perms, Perms.READ);
207      } else if (superUsers != null && ZKWatcher.isSuperUserId(superUsers, id)) {
208        // assert that super user has all the permissions
209        assertEquals(perms, Perms.ALL);
210      } else if (new Id("sasl", masterPrincipal).equals(id)) {
211        // hbase.master.kerberos.principal?
212        assertEquals(perms, Perms.ALL);
213      } else {
214        fail("An ACL is found which is not expected for the znode:" + znode + " , ACL:" + acl);
215      }
216    }
217  }
218
219  private void testFSPerms() throws IOException {
220    Path rootDir = FSUtils.getRootDir(conf);
221
222    LOG.info("");
223    LOG.info("***********************************************************************************");
224    LOG.info("Checking FS permissions for root dir:" + rootDir);
225    LOG.info("***********************************************************************************");
226    LOG.info("");
227    FileSystem fs = rootDir.getFileSystem(conf);
228
229    short expectedPerms = Short.valueOf(fsPerms, 8);
230
231    assertEquals(
232      FsPermission.createImmutable(expectedPerms),
233      fs.getFileStatus(rootDir).getPermission());
234
235    LOG.info("Checking FS permissions: SUCCESS");
236  }
237
238  public static void main(String[] args) throws Exception {
239    Configuration configuration = HBaseConfiguration.create();
240    IntegrationTestingUtility.setUseDistributedCluster(configuration);
241    IntegrationTestZKAndFSPermissions tool = new IntegrationTestZKAndFSPermissions();
242    int ret = ToolRunner.run(configuration, tool, args);
243    System.exit(ret);
244  }
245}