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