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.chaos.actions; 019 020import java.io.IOException; 021import java.io.InterruptedIOException; 022import java.security.PrivilegedExceptionAction; 023import org.apache.hadoop.conf.Configuration; 024import org.apache.hadoop.fs.FileSystem; 025import org.apache.hadoop.fs.Path; 026import org.apache.hadoop.hbase.util.CommonFSUtils; 027import org.apache.hadoop.hdfs.DistributedFileSystem; 028import org.apache.hadoop.security.UserGroupInformation; 029 030/** 031 * Configuration common across the HDFS Actions. 032 */ 033public final class HdfsActionUtils { 034 035 private HdfsActionUtils() { 036 } 037 038 /** 039 * Specify a user as whom HDFS actions should be run. The chaos process must have permissions 040 * sufficient to assume the role of the specified user. 041 * @see <a href= 042 * "https://hadoop.apache.org/docs/stable/hadoop-project-dist/hadoop-common/Superusers.html">Proxy 043 * user - Superusers Acting On Behalf Of Other Users</a> 044 */ 045 public static final String HDFS_USER_CONF_KEY = "org.apache.hadoop.hbase.chaos.actions.hdfs_user"; 046 047 private static DistributedFileSystem createUnproxiedDfs(final Configuration conf) 048 throws IOException { 049 final Path rootDir = CommonFSUtils.getRootDir(conf); 050 final FileSystem fs = rootDir.getFileSystem(conf); 051 return (DistributedFileSystem) fs; 052 } 053 054 /** 055 * Create an instance of {@link DistributedFileSystem} that honors {@value HDFS_USER_CONF_KEY}. 056 */ 057 static DistributedFileSystem createDfs(final Configuration conf) throws IOException { 058 final String proxyUser = conf.get(HDFS_USER_CONF_KEY); 059 if (proxyUser == null) { 060 return createUnproxiedDfs(conf); 061 } 062 final UserGroupInformation proxyUgi = 063 UserGroupInformation.createProxyUser(proxyUser, UserGroupInformation.getLoginUser()); 064 try { 065 return proxyUgi 066 .doAs((PrivilegedExceptionAction<DistributedFileSystem>) () -> createUnproxiedDfs(conf)); 067 } catch (InterruptedException e) { 068 final InterruptedIOException iioe = new InterruptedIOException(e.getMessage()); 069 iioe.setStackTrace(e.getStackTrace()); 070 throw iioe; 071 } 072 } 073}