blob: cc070f57873ab50ee258b9ed0d8e8206d047ad59 [file] [log] [blame]
/**
* Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership. The ASF licenses this file to you under the Apache License, Version
* 2.0 (the "License"); you may not use this file except in compliance with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the specific language governing permissions
* and limitations under the License.
*/
package org.apache.storm.security.auth.kerberos;
import java.security.Principal;
import java.util.Map;
import javax.security.auth.Subject;
import javax.security.auth.callback.CallbackHandler;
import javax.security.auth.kerberos.KerberosTicket;
import javax.security.auth.login.LoginException;
import javax.security.auth.spi.LoginModule;
import org.apache.storm.security.auth.ClientAuthUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* Custom LoginModule to enable Auto Login based on cached ticket.
*/
@SuppressWarnings("checkstyle:AbbreviationAsWordInName")
public class AutoTGTKrb5LoginModule implements LoginModule {
private static final Logger LOG = LoggerFactory.getLogger(AutoTGTKrb5LoginModule.class);
protected KerberosTicket kerbTicket = null;
// initial state
private Subject subject;
@Override
public void initialize(Subject subject,
CallbackHandler callbackHandler,
Map<String, ?> sharedState,
Map<String, ?> options) {
this.subject = subject;
}
@Override
public boolean login() throws LoginException {
LOG.debug("Acquire TGT from Cache");
getKerbTicketFromCache();
if (kerbTicket != null) {
return true;
} else {
throw new LoginException("Authentication failed, the TGT not found.");
}
}
protected void getKerbTicketFromCache() {
kerbTicket = AutoTGT.kerbTicket.get();
}
protected Principal getKerbTicketClient() {
if (kerbTicket != null) {
return kerbTicket.getClient();
}
return null;
}
@Override
public boolean commit() throws LoginException {
if (isSucceeded() == false) {
return false;
}
if (subject == null || subject.isReadOnly()) {
kerbTicket = null;
throw new LoginException("Authentication failed because the Subject is invalid.");
}
// Let us add the kerbClientPrinc and kerbTicket
// We need to clone the ticket because java.security.auth.kerberos assumes TGT is unique for each subject
// So, sharing TGT with multiple subjects can cause expired TGT to never refresh.
KerberosTicket kerbTicketCopy = ClientAuthUtils.cloneKerberosTicket(kerbTicket);
subject.getPrivateCredentials().add(kerbTicketCopy);
subject.getPrincipals().add(getKerbTicketClient());
LOG.debug("Commit Succeeded.");
return true;
}
@Override
public boolean abort() throws LoginException {
if (isSucceeded() == false) {
return false;
} else {
return logout();
}
}
@Override
public boolean logout() throws LoginException {
if (subject != null && !subject.isReadOnly() && kerbTicket != null) {
subject.getPrincipals().remove(kerbTicket.getClient());
AutoTGT.clearCredentials(subject, null);
}
kerbTicket = null;
return true;
}
private boolean isSucceeded() {
return kerbTicket != null;
}
}