Skip to content

HTTPS clone URL

Subversion checkout URL

You can clone with
or
.
Download ZIP
Browse files

FLUME-2749. Fix kerberos configuration error when using short names i…

…n multiple HDFS Sinks

(Johny Rufus via Hari)
  • Loading branch information...
commit a4946111383b3dfdb4c128fe5390ff3983213cbb 1 parent baca45a
@harishreedharan harishreedharan authored
View
6 flume-ng-auth/src/main/java/org/apache/flume/auth/FlumeAuthenticationUtil.java
@@ -17,6 +17,7 @@
*/
package org.apache.flume.auth;
+import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;
import org.apache.hadoop.security.SaslRpcServer;
import org.apache.hadoop.security.SecurityUtil;
@@ -90,6 +91,11 @@ public static CallbackHandler getSaslGssCallbackHandler() {
String resolvedPrinc = SecurityUtil.getServerPrincipal(principal, "");
return SaslRpcServer.splitKerberosName(resolvedPrinc);
}
+
+ @VisibleForTesting
+ static void clearCredentials() {
+ kerbAuthenticator = null;
+ }
}
View
7 flume-ng-auth/src/main/java/org/apache/flume/auth/KerberosAuthenticator.java
@@ -49,6 +49,7 @@
.getLogger(KerberosAuthenticator.class);
private volatile UserGroupInformation ugi;
+ private volatile KerberosUser prevUser;
private volatile PrivilegedExecutor privilegedExecutor;
private Map<String, PrivilegedExecutor> proxyCache = new HashMap<String, PrivilegedExecutor>();
@@ -128,10 +129,11 @@ public synchronized void authenticate(String principal, String keytab) {
// since we don't have to be unnecessarily protective if they switch all
// HDFS sinks to use a different principal all at once.
- Preconditions.checkState(ugi == null || ugi.getUserName().equals(resolvedPrincipal),
+ KerberosUser newUser = new KerberosUser(resolvedPrincipal, keytab);
+ Preconditions.checkState(prevUser == null || prevUser.equals(newUser),
"Cannot use multiple kerberos principals in the same agent. " +
" Must restart agent to use new principal or keytab. " +
- "Previous = %s, New = %s", ugi, resolvedPrincipal);
+ "Previous = %s, New = %s", prevUser, newUser);
// enable the kerberos mode of UGI, before doing anything else
@@ -173,6 +175,7 @@ public synchronized void authenticate(String principal, String keytab) {
"file ({})", new Object[] { resolvedPrincipal, keytab } );
UserGroupInformation.loginUserFromKeytab(resolvedPrincipal, keytab);
this.ugi = UserGroupInformation.getLoginUser();
+ this.prevUser = new KerberosUser(resolvedPrincipal, keytab);
this.privilegedExecutor = new UGIExecutor(this.ugi);
}
} catch (IOException e) {
View
70 flume-ng-auth/src/main/java/org/apache/flume/auth/KerberosUser.java
@@ -0,0 +1,70 @@
+/**
+ * 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.flume.auth;
+
+/**
+ * Simple Pair class used to define a unique (principal, keyTab) combination.
+ */
+public class KerberosUser {
+
+ private final String principal;
+ private final String keyTab;
+
+ public KerberosUser(String principal, String keyTab) {
+ this.principal = principal;
+ this.keyTab = keyTab;
+ }
+
+ public String getPrincipal() {
+ return principal;
+ }
+
+ public String getKeyTab() {
+ return keyTab;
+ }
+
+ @Override
+ public boolean equals(Object obj) {
+ if (obj == null) {
+ return false;
+ }
+ if (getClass() != obj.getClass()) {
+ return false;
+ }
+ final KerberosUser other = (KerberosUser) obj;
+ if ((this.principal == null) ? (other.principal != null) : !this.principal.equals(other.principal)) {
+ return false;
+ }
+ if ((this.keyTab == null) ? (other.keyTab != null) : !this.keyTab.equals(other.keyTab)) {
+ return false;
+ }
+ return true;
+ }
+
+ @Override
+ public int hashCode() {
+ int hash = 7;
+ hash = 41 * hash + (this.principal != null ? this.principal.hashCode() : 0);
+ hash = 41 * hash + (this.keyTab != null ? this.keyTab.hashCode() : 0);
+ return hash;
+ }
+
+ @Override
+ public String toString() {
+ return "{ principal: " + principal + ", keytab: " + keyTab + " }";
+ }
+}
View
37 flume-ng-auth/src/test/java/org/apache/flume/auth/TestFlumeAuthenticator.java
@@ -78,6 +78,8 @@ public void testFlumeLogin() throws IOException {
String keytab = flumeKeytab.getAbsolutePath();
String expResult = principal;
+ // Clear the previous statically stored logged in credentials
+ FlumeAuthenticationUtil.clearCredentials();
FlumeAuthenticator authenticator = FlumeAuthenticationUtil.getAuthenticator(
principal, keytab);
assertTrue(authenticator.isAuthenticated());
@@ -125,4 +127,39 @@ public void testProxyAs() throws IOException {
assertEquals("Proxy as didn't generate the expected username", expResult, result);
}
+ @Test
+ public void testFlumeLoginPrincipalWithoutRealm() throws Exception {
+ String principal = "flume";
+ File keytab = new File(workDir, "flume2.keytab");
+ kdc.createPrincipal(keytab, principal);
+ String expResult = principal+"@" + kdc.getRealm();
+
+ // Clear the previous statically stored logged in credentials
+ FlumeAuthenticationUtil.clearCredentials();
+
+ FlumeAuthenticator authenticator = FlumeAuthenticationUtil.getAuthenticator(
+ principal, keytab.getAbsolutePath());
+ assertTrue(authenticator.isAuthenticated());
+
+ String result = ((KerberosAuthenticator)authenticator).getUserName();
+ assertEquals("Initial login failed", expResult, result);
+
+ authenticator = FlumeAuthenticationUtil.getAuthenticator(
+ principal, keytab.getAbsolutePath());
+ result = ((KerberosAuthenticator)authenticator).getUserName();
+ assertEquals("Re-login failed", expResult, result);
+
+ principal = "alice";
+ keytab = aliceKeytab;
+ try {
+ authenticator = FlumeAuthenticationUtil.getAuthenticator(
+ principal, keytab.getAbsolutePath());
+ result = ((KerberosAuthenticator)authenticator).getUserName();
+ fail("Login should have failed with a new principal: " + result);
+ } catch (Exception ex) {
+ assertTrue("Login with a new principal failed, but for an unexpected "
+ + "reason: " + ex.getMessage(),
+ ex.getMessage().contains("Cannot use multiple kerberos principals"));
+ }
+ }
}
Please sign in to comment.
Something went wrong with that request. Please try again.