You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@kudu.apache.org by "Todd Lipcon (JIRA)" <ji...@apache.org> on 2018/01/31 02:34:00 UTC

[jira] [Reopened] (KUDU-2259) kudu-spark imports authentication token into client multiple times

     [ https://issues.apache.org/jira/browse/KUDU-2259?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Todd Lipcon reopened KUDU-2259:
-------------------------------

> kudu-spark imports authentication token into client multiple times
> ------------------------------------------------------------------
>
>                 Key: KUDU-2259
>                 URL: https://issues.apache.org/jira/browse/KUDU-2259
>             Project: Kudu
>          Issue Type: Bug
>          Components: spark
>    Affects Versions: 1.6.0
>            Reporter: Will Berkeley
>            Priority: Major
>             Fix For: 1.7.0
>
>
> kudu-spark should have one KuduContext per task, which is sent serialized from the driver with an authentication token. The KuduContext either retrieves a Kudu client from a JVM-scoped cache, or creates one and puts it in the cache, and finally imports its authentication token into the client.
> Under default configuration in an un-Kerberized cluster, the client uses the authentication token to connect to the cluster. However, if -rpc_encryption=disabled, then the client will not use the authentication token. This causes the master to issue an authentication token to the client, and the new token replaces the old token in the client.
> While there's one KuduContext per task, multiple tasks may run on the same executor. If this occurs, each KuduContext tries to import its authentication token into the client. If the client has already received a token from the master because encryption is disabled, then it's possible that the KuduContext's token and the master-issued token are for different users, since the KuduContext's token was issued on the driver to the driver's Unix user and the master-issued token is issued to the executor's user.
> An example of the exception that occurred when running spark2-shell as root:
> {noformat}
> 18/01/11 12:14:01 WARN scheduler.TaskSetManager: Lost task 0.0 in stage 1.0 (TID 1, kudu-tserver-01, executor 1): java.lang.IllegalArgumentException: cannot import authentication data from a different user: old='yarn', new='root'
> 	at org.apache.kudu.client.SecurityContext.checkUserMatches(SecurityContext.java:128)
>   	at org.apache.kudu.client.SecurityContext.importAuthenticationCredentials(SecurityContext.java:138)
> 	at org.apache.kudu.client.AsyncKuduClient.importAuthenticationCredentials(AsyncKuduClient.java:677)
>   	at org.apache.kudu.spark.kudu.KuduContext.asyncClient$lzycompute(KuduContext.scala:103)
> 	at org.apache.kudu.spark.kudu.KuduContext.asyncClient(KuduContext.scala:100)
>   	at org.apache.kudu.spark.kudu.KuduContext.syncClient$lzycompute(KuduContext.scala:98)
> 	at org.apache.kudu.spark.kudu.KuduContext.syncClient(KuduContext.scala:98)
>   	at org.apache.kudu.spark.kudu.KuduRDD.compute(KuduRDD.scala:71)
> 	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:323)
>   	at org.apache.spark.rdd.RDD.iterator(RDD.scala:287)
> 	at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
>   	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:323)
> 	at org.apache.spark.rdd.RDD.iterator(RDD.scala:287)
>   	at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
> 	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:323)
>   	at org.apache.spark.rdd.RDD.iterator(RDD.scala:287)
> 	at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
>   	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:323)
> 	at org.apache.spark.rdd.RDD.iterator(RDD.scala:287)
>   	at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
> 	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:323)
>   	at org.apache.spark.rdd.RDD.iterator(RDD.scala:287)
> 	at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:87)
>   	at org.apache.spark.scheduler.Task.run(Task.scala:108)
> 	at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:338)
>   	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
> 	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
>   	at java.lang.Thread.run(Thread.java:748)
> {noformat}



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