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.security.token; 019 020import static org.apache.hadoop.hdfs.protocol.HdfsConstants.HDFS_URI_SCHEME; 021import static org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier.HDFS_DELEGATION_KIND; 022import static org.apache.hadoop.hdfs.web.WebHdfsConstants.SWEBHDFS_SCHEME; 023import static org.apache.hadoop.hdfs.web.WebHdfsConstants.SWEBHDFS_TOKEN_KIND; 024import static org.apache.hadoop.hdfs.web.WebHdfsConstants.WEBHDFS_SCHEME; 025import static org.apache.hadoop.hdfs.web.WebHdfsConstants.WEBHDFS_TOKEN_KIND; 026 027import java.io.IOException; 028import java.util.Objects; 029import org.apache.hadoop.fs.FileSystem; 030import org.apache.hadoop.hbase.security.UserProvider; 031import org.apache.hadoop.security.token.Token; 032import org.apache.yetus.audience.InterfaceAudience; 033import org.apache.yetus.audience.InterfaceStability; 034import org.slf4j.Logger; 035import org.slf4j.LoggerFactory; 036 037/** 038 * Helper class to obtain a filesystem delegation token. Mainly used by Map-Reduce jobs that 039 * requires to read/write data to a remote file-system (e.g. BulkLoad, ExportSnapshot). 040 */ 041@InterfaceAudience.Private 042@InterfaceStability.Evolving 043public class FsDelegationToken { 044 private static final Logger LOG = LoggerFactory.getLogger(FsDelegationToken.class); 045 046 private final UserProvider userProvider; 047 private final String renewer; 048 049 private boolean hasForwardedToken = false; 050 private Token<?> userToken = null; 051 private FileSystem fs = null; 052 053 /* 054 * @param renewer the account name that is allowed to renew the token. 055 */ 056 public FsDelegationToken(final UserProvider userProvider, final String renewer) { 057 this.userProvider = userProvider; 058 this.renewer = renewer; 059 } 060 061 /** 062 * Acquire the delegation token for the specified filesystem. Before requesting a new delegation 063 * token, tries to find one already available. Currently supports checking existing delegation 064 * tokens for swebhdfs, webhdfs and hdfs. 065 * @param fs the filesystem that requires the delegation token 066 * @throws IOException on fs.getDelegationToken() failure 067 */ 068 public void acquireDelegationToken(final FileSystem fs) throws IOException { 069 String tokenKind; 070 String scheme = fs.getUri().getScheme(); 071 if (SWEBHDFS_SCHEME.equalsIgnoreCase(scheme)) { 072 tokenKind = SWEBHDFS_TOKEN_KIND.toString(); 073 } else if (WEBHDFS_SCHEME.equalsIgnoreCase(scheme)) { 074 tokenKind = WEBHDFS_TOKEN_KIND.toString(); 075 } else if (HDFS_URI_SCHEME.equalsIgnoreCase(scheme)) { 076 tokenKind = HDFS_DELEGATION_KIND.toString(); 077 } else { 078 LOG.warn("Unknown FS URI scheme: " + scheme); 079 // Preserve default behavior 080 tokenKind = HDFS_DELEGATION_KIND.toString(); 081 } 082 083 acquireDelegationToken(tokenKind, fs); 084 } 085 086 /** 087 * Acquire the delegation token for the specified filesystem and token kind. Before requesting a 088 * new delegation token, tries to find one already available. 089 * @param tokenKind non-null token kind to get delegation token from the {@link UserProvider} 090 * @param fs the filesystem that requires the delegation token 091 * @throws IOException on fs.getDelegationToken() failure 092 */ 093 public void acquireDelegationToken(final String tokenKind, final FileSystem fs) 094 throws IOException { 095 Objects.requireNonNull(tokenKind, "tokenKind:null"); 096 if (userProvider.isHadoopSecurityEnabled()) { 097 this.fs = fs; 098 userToken = userProvider.getCurrent().getToken(tokenKind, fs.getCanonicalServiceName()); 099 if (userToken == null) { 100 hasForwardedToken = false; 101 userToken = fs.getDelegationToken(renewer); 102 } else { 103 hasForwardedToken = true; 104 LOG.info("Use the existing token: " + userToken); 105 } 106 } 107 } 108 109 /** 110 * Releases a previously acquired delegation token. 111 */ 112 public void releaseDelegationToken() { 113 if (userProvider.isHadoopSecurityEnabled()) { 114 if (userToken != null && !hasForwardedToken) { 115 try { 116 userToken.cancel(this.fs.getConf()); 117 } catch (Exception e) { 118 LOG.warn("Failed to cancel HDFS delegation token: " + userToken, e); 119 } 120 } 121 this.userToken = null; 122 this.fs = null; 123 } 124 } 125 126 public UserProvider getUserProvider() { 127 return userProvider; 128 } 129 130 /** Returns the account name that is allowed to renew the token. */ 131 public String getRenewer() { 132 return renewer; 133 } 134 135 /** Returns the delegation token acquired, or null in case it was not acquired */ 136 public Token<?> getUserToken() { 137 return userToken; 138 } 139 140 public FileSystem getFileSystem() { 141 return fs; 142 } 143}