[ 
https://issues.apache.org/jira/browse/KAFKA-7119?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16580070#comment-16580070
 ] 

ASF GitHub Bot commented on KAFKA-7119:
---------------------------------------

rajinisivaram closed pull request #5487: KAFKA-7119: Handle transient Kerberos 
errors as non-fatal exceptions
URL: https://github.com/apache/kafka/pull/5487
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git 
a/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslClientAuthenticator.java
 
b/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslClientAuthenticator.java
index 2ef6d77f13f..8d6549d867c 100644
--- 
a/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslClientAuthenticator.java
+++ 
b/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslClientAuthenticator.java
@@ -42,6 +42,7 @@
 import org.apache.kafka.common.requests.SaslHandshakeResponse;
 import org.apache.kafka.common.security.auth.AuthenticateCallbackHandler;
 import org.apache.kafka.common.security.auth.KafkaPrincipal;
+import org.apache.kafka.common.utils.Java;
 import org.apache.kafka.common.utils.Utils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -51,6 +52,7 @@
 import javax.security.sasl.SaslClient;
 import javax.security.sasl.SaslException;
 import java.io.IOException;
+import java.lang.reflect.Method;
 import java.nio.ByteBuffer;
 import java.nio.channels.SelectionKey;
 import java.security.Principal;
@@ -360,11 +362,9 @@ public void close() throws IOException {
                 });
         } catch (PrivilegedActionException e) {
             String error = "An error: (" + e + ") occurred when evaluating 
SASL token received from the Kafka Broker.";
+            KerberosError kerberosError = KerberosError.fromException(e);
             // Try to provide hints to use about what went wrong so they can 
fix their configuration.
-            // TODO: introspect about e: look for GSS information.
-            final String unknownServerErrorText =
-                "(Mechanism level: Server not found in Kerberos database (7) - 
UNKNOWN_SERVER)";
-            if (e.toString().contains(unknownServerErrorText)) {
+            if (kerberosError == KerberosError.SERVER_NOT_FOUND) {
                 error += " This may be caused by Java's being unable to 
resolve the Kafka Broker's" +
                     " hostname correctly. You may want to try to adding" +
                     " '-Dsun.net.spi.nameservice.provider.1=dns,sun' to your 
client's JVMFLAGS environment." +
@@ -373,7 +373,13 @@ public void close() throws IOException {
             }
             error += " Kafka Client will go to AUTHENTICATION_FAILED state.";
             //Unwrap the SaslException inside `PrivilegedActionException`
-            throw new SaslAuthenticationException(error, e.getCause());
+            Throwable cause = e.getCause();
+            // Treat transient Kerberos errors as non-fatal SaslExceptions 
that are processed as I/O exceptions
+            // and all other failures as fatal SaslAuthenticationException.
+            if (kerberosError != null && kerberosError.retriable)
+                throw new SaslException(error, cause);
+            else
+                throw new SaslAuthenticationException(error, cause);
         }
     }
 
@@ -436,4 +442,74 @@ static final String firstPrincipal(Subject subject) {
                 throw new KafkaException("Principal could not be determined 
from Subject, this may be a transient failure due to Kerberos re-login");
         }
     }
+
+    /**
+     * Kerberos exceptions that may require special handling. The standard 
Kerberos error codes
+     * for these errors are retrieved using KrbException#errorCode() from the 
underlying Kerberos
+     * exception thrown during {@link SaslClient#evaluateChallenge(byte[])}.
+     */
+    private enum KerberosError {
+        // (Mechanism level: Server not found in Kerberos database (7) - 
UNKNOWN_SERVER)
+        // This is retriable, but included here to add extra logging for this 
case.
+        SERVER_NOT_FOUND(7, false),
+        // (Mechanism level: Client not yet valid - try again later (21))
+        CLIENT_NOT_YET_VALID(21, true),
+        // (Mechanism level: Ticket not yet valid (33) - Ticket not yet 
valid)])
+        // This could be a small timing window.
+        TICKET_NOT_YET_VALID(33, true),
+        // (Mechanism level: Request is a replay (34) - Request is a replay)
+        // Replay detection used to prevent DoS attacks can result in false 
positives, so retry on error.
+        REPLAY(34, true);
+
+
+        private static final Class<?> KRB_EXCEPTION_CLASS;
+        private static final Method KRB_EXCEPTION_RETURN_CODE_METHOD;
+
+        static {
+            try {
+                if (Java.isIbmJdk()) {
+                    KRB_EXCEPTION_CLASS = 
Class.forName("com.ibm.security.krb5.internal.KrbException");
+                } else {
+                    KRB_EXCEPTION_CLASS = 
Class.forName("sun.security.krb5.KrbException");
+                }
+                KRB_EXCEPTION_RETURN_CODE_METHOD = 
KRB_EXCEPTION_CLASS.getMethod("returnCode");
+            } catch (Exception e) {
+                throw new KafkaException("Kerberos exceptions could not be 
initialized", e);
+            }
+        }
+
+        private final int errorCode;
+        private final boolean retriable;
+
+        KerberosError(int errorCode, boolean retriable) {
+            this.errorCode = errorCode;
+            this.retriable = retriable;
+        }
+
+        private static KerberosError fromException(Exception exception) {
+            Throwable cause = exception.getCause();
+            while (cause != null && !KRB_EXCEPTION_CLASS.isInstance(cause)) {
+                cause = cause.getCause();
+            }
+            if (cause == null)
+                return null;
+            else {
+                try {
+                    Integer errorCode = (Integer) 
KRB_EXCEPTION_RETURN_CODE_METHOD.invoke(cause);
+                    return fromErrorCode(errorCode);
+                } catch (Exception e) {
+                    LOG.trace("Kerberos return code could not be determined 
from {} due to {}", exception, e);
+                    return null;
+                }
+            }
+        }
+
+        private static KerberosError fromErrorCode(int errorCode) {
+            for (KerberosError error : values()) {
+                if (error.errorCode == errorCode)
+                    return error;
+            }
+            return null;
+        }
+    }
 }
diff --git 
a/core/src/test/scala/integration/kafka/server/GssapiAuthenticationTest.scala 
b/core/src/test/scala/integration/kafka/server/GssapiAuthenticationTest.scala
new file mode 100644
index 00000000000..037f61408a5
--- /dev/null
+++ 
b/core/src/test/scala/integration/kafka/server/GssapiAuthenticationTest.scala
@@ -0,0 +1,155 @@
+/**
+  * 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 kafka.server
+
+import java.net.InetSocketAddress
+import java.util.Properties
+import java.util.concurrent.{Executors, TimeUnit}
+
+import kafka.api.{Both, IntegrationTestHarness, SaslSetup}
+import kafka.utils.TestUtils
+import org.apache.kafka.clients.CommonClientConfigs
+import org.apache.kafka.common.config.SaslConfigs
+import org.apache.kafka.common.network._
+import org.apache.kafka.common.security.{JaasContext, TestSecurityConfig}
+import org.apache.kafka.common.security.auth.SecurityProtocol
+import org.junit.Assert._
+import org.junit.{After, Before, Test}
+
+class GssapiAuthenticationTest extends IntegrationTestHarness with SaslSetup {
+
+  override val producerCount = 0
+  override val consumerCount = 0
+  override val serverCount = 1
+  override protected def securityProtocol = SecurityProtocol.SASL_PLAINTEXT
+
+  private val kafkaClientSaslMechanism = "GSSAPI"
+  private val kafkaServerSaslMechanisms = List("GSSAPI")
+
+  private val numThreads = 10
+  private val executor = Executors.newFixedThreadPool(numThreads)
+  private val clientConfig: Properties = new Properties
+  private var serverAddr: InetSocketAddress = _
+
+  @Before
+  override def setUp() {
+    startSasl(jaasSections(kafkaServerSaslMechanisms, 
Option(kafkaClientSaslMechanism), Both))
+    super.setUp()
+    serverAddr = new InetSocketAddress("localhost",
+      
servers.head.boundPort(ListenerName.forSecurityProtocol(SecurityProtocol.SASL_PLAINTEXT)))
+
+    clientConfig.put(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG, 
SecurityProtocol.SASL_PLAINTEXT.name)
+    clientConfig.put(SaslConfigs.SASL_MECHANISM, kafkaClientSaslMechanism)
+    clientConfig.put(SaslConfigs.SASL_JAAS_CONFIG, 
jaasClientLoginModule(kafkaClientSaslMechanism))
+    clientConfig.put(CommonClientConfigs.CONNECTIONS_MAX_IDLE_MS_CONFIG, 
"5000")
+  }
+
+  @After
+  override def tearDown(): Unit = {
+    executor.shutdownNow()
+    super.tearDown()
+    closeSasl()
+  }
+
+  /**
+   * Tests that Kerberos replay error `Request is a replay (34)` is not 
handled as an authentication exception
+   * since replay detection used to detect DoS attacks may occasionally reject 
valid concurrent requests.
+   */
+  @Test
+  def testRequestIsAReplay(): Unit = {
+    val successfulAuthsPerThread = 10
+    val futures = (0 until numThreads).map(_ => executor.submit(new Runnable {
+      override def run(): Unit = 
verifyRetriableFailuresDuringAuthentication(successfulAuthsPerThread)
+    }))
+    futures.foreach(_.get(60, TimeUnit.SECONDS))
+    assertEquals(0, TestUtils.totalMetricValue(servers.head, 
"failed-authentication-total"))
+    val successfulAuths = TestUtils.totalMetricValue(servers.head, 
"successful-authentication-total")
+    assertTrue("Too few authentications: " + successfulAuths, successfulAuths 
> successfulAuthsPerThread * numThreads)
+  }
+
+  /**
+   * Tests that Kerberos error `Server not found in Kerberos database (7)` is 
handled
+   * as a fatal authentication failure.
+   */
+  @Test
+  def testServerNotFoundInKerberosDatabase(): Unit = {
+    val jaasConfig = clientConfig.getProperty(SaslConfigs.SASL_JAAS_CONFIG)
+    val invalidServiceConfig = jaasConfig.replace("serviceName=\"kafka\"", 
"serviceName=\"invalid-service\"")
+    clientConfig.put(SaslConfigs.SASL_JAAS_CONFIG, invalidServiceConfig)
+    clientConfig.put(SaslConfigs.SASL_KERBEROS_SERVICE_NAME, "invalid-service")
+    verifyNonRetriableAuthenticationFailure()
+  }
+
+  /**
+   * Verifies that any exceptions during authentication with the current 
`clientConfig` are
+   * notified with disconnect state `AUTHENTICATE` (and not 
`AUTHENTICATION_FAILED`). This
+   * is to ensure that NetworkClient doesn't handle this as a fatal 
authentication failure,
+   * but as a transient I/O exception. So Producer/Consumer/AdminClient will 
retry
+   * any operation based on their configuration until timeout and will not 
propagate
+   * the exception to the application.
+   */
+  private def verifyRetriableFailuresDuringAuthentication(numSuccessfulAuths: 
Int): Unit = {
+    val selector = createSelector()
+    try {
+      var actualSuccessfulAuths = 0
+      while (actualSuccessfulAuths < numSuccessfulAuths) {
+        val nodeId = actualSuccessfulAuths.toString
+        selector.connect(nodeId, serverAddr, 1024, 1024)
+        TestUtils.waitUntilTrue(() => {
+          selector.poll(100)
+          val disconnectState = selector.disconnected().get(nodeId)
+          // Verify that disconnect state is not AUTHENTICATION_FAILED
+          if (disconnectState != null)
+            assertEquals(ChannelState.State.AUTHENTICATE, 
disconnectState.state())
+          selector.isChannelReady(nodeId) || disconnectState != null
+        }, "Client not ready or disconnected within timeout")
+        if (selector.isChannelReady(nodeId))
+          actualSuccessfulAuths += 1
+        selector.close(nodeId)
+      }
+    } finally {
+      selector.close()
+    }
+  }
+
+  /**
+   * Verifies that authentication with the current `clientConfig` results in 
disconnection and that
+   * the disconnection is notified with disconnect state 
`AUTHENTICATION_FAILED`. This is to ensure
+   * that NetworkClient handles this as a fatal authentication failure that is 
propagated to
+   * applications by Producer/Consumer/AdminClient without retrying and 
waiting for timeout.
+   */
+  private def verifyNonRetriableAuthenticationFailure(): Unit = {
+    val selector = createSelector()
+    val nodeId = "1"
+    selector.connect(nodeId, serverAddr, 1024, 1024)
+    TestUtils.waitUntilTrue(() => {
+      selector.poll(100)
+      val disconnectState = selector.disconnected().get(nodeId)
+      if (disconnectState != null)
+        assertEquals(ChannelState.State.AUTHENTICATION_FAILED, 
disconnectState.state())
+      disconnectState != null
+    }, "Client not disconnected within timeout")
+  }
+
+  private def createSelector(): Selector = {
+    val channelBuilder = ChannelBuilders.clientChannelBuilder(securityProtocol,
+      JaasContext.Type.CLIENT, new TestSecurityConfig(clientConfig), null, 
kafkaClientSaslMechanism, true)
+    NetworkTestUtils.createSelector(channelBuilder)
+  }
+}
diff --git 
a/core/src/test/scala/integration/kafka/server/ScramServerStartupTest.scala 
b/core/src/test/scala/integration/kafka/server/ScramServerStartupTest.scala
index 18b4f8e23b8..85028744ca3 100644
--- a/core/src/test/scala/integration/kafka/server/ScramServerStartupTest.scala
+++ b/core/src/test/scala/integration/kafka/server/ScramServerStartupTest.scala
@@ -58,16 +58,9 @@ class ScramServerStartupTest extends IntegrationTestHarness 
with SaslSetup {
 
   @Test
   def testAuthentications(): Unit = {
-    val successfulAuths = 
totalAuthentications("successful-authentication-total")
+    val successfulAuths = TestUtils.totalMetricValue(servers.head, 
"successful-authentication-total")
     assertTrue("No successful authentications", successfulAuths > 0)
-    val failedAuths = totalAuthentications("failed-authentication-total")
+    val failedAuths = TestUtils.totalMetricValue(servers.head, 
"failed-authentication-total")
     assertEquals(0, failedAuths)
   }
-
-  private def totalAuthentications(metricName: String): Int = {
-    val allMetrics = servers.head.metrics.metrics
-    val totalAuthCount = 
allMetrics.values().asScala.filter(_.metricName().name() == metricName)
-      .foldLeft(0.0)((total, metric) => total + 
metric.metricValue.asInstanceOf[Double])
-    totalAuthCount.toInt
-  }
 }
diff --git a/core/src/test/scala/unit/kafka/utils/TestUtils.scala 
b/core/src/test/scala/unit/kafka/utils/TestUtils.scala
index 978362782ac..bdd68820b32 100755
--- a/core/src/test/scala/unit/kafka/utils/TestUtils.scala
+++ b/core/src/test/scala/unit/kafka/utils/TestUtils.scala
@@ -1415,4 +1415,11 @@ object TestUtils extends Logging {
             cause.getClass().getName, clazz.isInstance(cause))
     }
   }
+
+  def totalMetricValue(server: KafkaServer, metricName: String): Long = {
+    val allMetrics = server.metrics.metrics
+    val total = allMetrics.values().asScala.filter(_.metricName().name() == 
metricName)
+      .foldLeft(0.0)((total, metric) => total + 
metric.metricValue.asInstanceOf[Double])
+    total.toLong
+  }
 }


 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


> Intermittent test failure with GSSAPI authentication failure
> ------------------------------------------------------------
>
>                 Key: KAFKA-7119
>                 URL: https://issues.apache.org/jira/browse/KAFKA-7119
>             Project: Kafka
>          Issue Type: Bug
>          Components: security
>    Affects Versions: 2.0.0
>            Reporter: Rajini Sivaram
>            Assignee: Rajini Sivaram
>            Priority: Major
>
> I have seen this failure a couple of times in builds (e.g. 
> [https://builds.apache.org/job/kafka-pr-jdk10-scala2.12/2412/testReport/junit/kafka.api/SaslSslAdminClientIntegrationTest/testLogStartOffsetCheckpoint/)]
> {quote}
> org.apache.kafka.common.errors.SaslAuthenticationException: An error: 
> (java.security.PrivilegedActionException: javax.security.sasl.SaslException: 
> GSS initiate failed [Caused by GSSException: No valid credentials provided 
> (Mechanism level: Request is a replay (34) - Request is a replay)]) occurred 
> when evaluating SASL token received from the Kafka Broker. Kafka Client will 
> go to AUTHENTICATION_FAILED state. Caused by: 
> javax.security.sasl.SaslException: GSS initiate failed [Caused by 
> GSSException: No valid credentials provided (Mechanism level: Request is a 
> replay (34) - Request is a replay)] at 
> jdk.security.jgss/com.sun.security.sasl.gsskerb.GssKrb5Client.evaluateChallenge(GssKrb5Client.java:211)
>  at 
> org.apache.kafka.common.security.authenticator.SaslClientAuthenticator$2.run(SaslClientAuthenticator.java:358)
>  at 
> org.apache.kafka.common.security.authenticator.SaslClientAuthenticator$2.run(SaslClientAuthenticator.java:356)
>  at java.base/java.security.AccessController.doPrivileged(Native Method) at 
> java.base/javax.security.auth.Subject.doAs(Subject.java:423) at 
> org.apache.kafka.common.security.authenticator.SaslClientAuthenticator.createSaslToken(SaslClientAuthenticator.java:356)
>  at 
> org.apache.kafka.common.security.authenticator.SaslClientAuthenticator.sendSaslClientToken(SaslClientAuthenticator.java:268)
>  at 
> org.apache.kafka.common.security.authenticator.SaslClientAuthenticator.authenticate(SaslClientAuthenticator.java:205)
>  at 
> org.apache.kafka.common.network.KafkaChannel.prepare(KafkaChannel.java:127) 
> at 
> org.apache.kafka.common.network.Selector.pollSelectionKeys(Selector.java:487) 
> at org.apache.kafka.common.network.Selector.poll(Selector.java:425) at 
> org.apache.kafka.clients.NetworkClient.poll(NetworkClient.java:510) at 
> org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient.poll(ConsumerNetworkClient.java:271)
>  at 
> org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient.poll(ConsumerNetworkClient.java:242)
>  at 
> org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient.poll(ConsumerNetworkClient.java:218)
>  at 
> org.apache.kafka.clients.consumer.internals.AbstractCoordinator.ensureCoordinatorReady(AbstractCoordinator.java:231)
>  at 
> org.apache.kafka.clients.consumer.internals.ConsumerCoordinator.poll(ConsumerCoordinator.java:314)
>  at 
> org.apache.kafka.clients.consumer.KafkaConsumer.updateAssignmentMetadataIfNeeded(KafkaConsumer.java:1218)
>  at 
> org.apache.kafka.clients.consumer.KafkaConsumer.poll(KafkaConsumer.java:1181) 
> at 
> org.apache.kafka.clients.consumer.KafkaConsumer.poll(KafkaConsumer.java:1115) 
> at 
> kafka.api.AdminClientIntegrationTest.$anonfun$subscribeAndWaitForAssignment$2(AdminClientIntegrationTest.scala:980)
>  at kafka.utils.TestUtils$.waitUntilTrue(TestUtils.scala:781) at 
> kafka.api.AdminClientIntegrationTest.subscribeAndWaitForAssignment(AdminClientIntegrationTest.scala:979)
>  at 
> kafka.api.AdminClientIntegrationTest.testLogStartOffsetCheckpoint(AdminClientIntegrationTest.scala:755)
>  at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native 
> Method) at 
> java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
>  at 
> java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
>  at java.base/java.lang.reflect.Method.invoke(Method.java:564) at 
> org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:50)
>  at 
> org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12)
>  at 
> org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:47)
>  at 
> org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17)
>  at 
> org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26) 
> at 
> org.junit.internal.runners.statements.RunAfters.evaluate(RunAfters.java:27) 
> at 
> org.junit.internal.runners.statements.FailOnTimeout$CallableStatement.call(FailOnTimeout.java:298)
>  at 
> org.junit.internal.runners.statements.FailOnTimeout$CallableStatement.call(FailOnTimeout.java:292)
>  at java.base/java.util.concurrent.FutureTask.run(FutureTask.java:264) at 
> java.base/java.lang.Thread.run(Thread.java:844) Caused by: GSSException: No 
> valid credentials provided (Mechanism level: Request is a replay (34) - 
> Request is a replay) at 
> java.security.jgss/sun.security.jgss.krb5.Krb5Context.initSecContext(Krb5Context.java:771)
>  at 
> java.security.jgss/sun.security.jgss.GSSContextImpl.initSecContext(GSSContextImpl.java:265)
>  at 
> java.security.jgss/sun.security.jgss.GSSContextImpl.initSecContext(GSSContextImpl.java:196)
>  at 
> jdk.security.jgss/com.sun.security.sasl.gsskerb.GssKrb5Client.evaluateChallenge(GssKrb5Client.java:192)
>  ... 37 more Caused by: KrbException: Request is a replay (34) - Request is a 
> replay at 
> java.security.jgss/sun.security.krb5.KrbTgsRep.<init>(KrbTgsRep.java:73) at 
> java.security.jgss/sun.security.krb5.KrbTgsReq.getReply(KrbTgsReq.java:251) 
> at 
> java.security.jgss/sun.security.krb5.KrbTgsReq.sendAndGetCreds(KrbTgsReq.java:262)
>  at 
> java.security.jgss/sun.security.krb5.internal.CredentialsUtil.serviceCreds(CredentialsUtil.java:308)
>  at 
> java.security.jgss/sun.security.krb5.internal.CredentialsUtil.acquireServiceCreds(CredentialsUtil.java:126)
>  at 
> java.security.jgss/sun.security.krb5.Credentials.acquireServiceCreds(Credentials.java:458)
>  at 
> java.security.jgss/sun.security.jgss.krb5.Krb5Context.initSecContext(Krb5Context.java:695)
>  ... 40 more Caused by: KrbException: Identifier doesn't match expected value 
> (906) at 
> java.security.jgss/sun.security.krb5.internal.KDCRep.init(KDCRep.java:140) at 
> java.security.jgss/sun.security.krb5.internal.TGSRep.init(TGSRep.java:65) at 
> java.security.jgss/sun.security.krb5.internal.TGSRep.<init>(TGSRep.java:60) 
> at java.security.jgss/sun.security.krb5.KrbTgsRep.<init>(KrbTgsRep.java:55)
> ... 46 more
> {quote}
>  
>  
>  



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to