The submit command: spark-submit \ --master yarn \ --deploy-mode cluster \ --conf "spark.executor.extraJavaOptions=-Djava.security.auth.login.config=kafka_client_jaas.conf -Dlog4j.configuration=xxx -Djava.util.Arrays.useLegacyMergeSort=true" \ --conf "spark.driver.extraJavaOptions=-Djava.security.auth.login.config=kafka_client_jaas.conf -Dlog4j.configuration=xxx -Djava.util.Arrays.useLegacyMergeSort=true" \ --conf spark.ui.port=18086 \ --conf spark.executor.memory=${executor_memory} \ --conf spark.executor.instances=${num_executors} \ --conf spark.executor.cores=${executor_cores} \ --conf spark.driver.memory=4g \ --conf spark.driver.maxResultSize=3g \ --conf spark.kafka.broker.ingest=xxx \ --conf spark.kafka.zookeeper.ingest=xxx \ --conf spark.kafka.broker.egest=xxx \ --conf spark.kafka.topic.input=xxx \ --conf spark.kafka.topic.output=xxx \ --conf spark.kafka.input.interval=10 \ --conf spark.kafka.group=xxx \ --conf spark.streaming.kafka.maxRetries=10 \ --conf spark.kafka.security.protocol.ingress=SASL_PLAINTEXT \ --conf spark.kafka.security.protocol.egress=SASL_PLAINTEXT \ --conf spark.fetch.message.max.bytes=104857600 \ --conf spark.hive.enable.stats=true \ --conf spark.streaming.backpressure.enabled=true \ --conf spark.streaming.kafka.maxRatePerPartition=1 \ --conf spark.streaming.receiver.maxRate=10 \ --conf spark.executor.heartbeatInterval=120s \ --conf spark.network.timeout=600s \ --conf spark.yarn.scheduler.heartbeat.interval-ms=1000 \ --conf spark.sql.parquet.compression.codec=snappy \ --conf spark.scheduler.minRegisteredResourcesRatio=1 \ --conf spark.yarn.maxAppAttempts=10 \ --conf spark.yarn.am.attemptFailuresValidityInterval=1h \ --conf spark.yarn.max.executor.failures=$((8 * ${num_executors})) `# Increase max executor failures (Default: max(numExecutors * 2, 3))` \ --conf spark.yarn.executor.failuresValidityInterval=1h \ --conf spark.task.maxFailures=8 \ --conf spark.yarn.submit.waitAppCompletion=false \ --conf spark.yarn.principal=xxx \ --conf spark.yarn.keytab=xxx \ --conf spark.hadoop.fs.hdfs.impl.disable.cache=true \ --queue default \ ${APP_HOME}/xxx.jar
The stack trace: WARN Client: Exception encountered while connecting to the server : org.apache.hadoop.ipc.RemoteException(org.apache.hadoop.security.token.Secret Manager$InvalidToken): token (HDFS_DELEGATION_TOKEN token 155456 for spark) can't be found in cache Exception in thread "main" org.apache.hadoop.ipc.RemoteException(org.apache.hadoop.security.token.SecretManager$InvalidToken): token (HDFS_DELEGATION_TOKEN token 1 55456 for spark) can't be found in cache at org.apache.hadoop.ipc.Client.getRpcResponse(Client.java:1554) at org.apache.hadoop.ipc.Client.call(Client.java:1498) at org.apache.hadoop.ipc.Client.call(Client.java:1398) at org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:233) at com.sun.proxy.$Proxy10.getFileInfo(Unknown Source) at org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolTranslatorPB.getFileInfo(ClientNamenodeProtocolTranslatorPB.java:818) at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) at java.lang.reflect.Method.invoke(Method.java:498) at org.apache.hadoop.io.retry.RetryInvocationHandler.invokeMethod(RetryInvocationHandler.java:291) at org.apache.hadoop.io.retry.RetryInvocationHandler.invoke(RetryInvocationHandler.java:203) at org.apache.hadoop.io.retry.RetryInvocationHandler.invoke(RetryInvocationHandler.java:185) at com.sun.proxy.$Proxy11.getFileInfo(Unknown Source) at org.apache.hadoop.hdfs.DFSClient.getFileInfo(DFSClient.java:2165) at org.apache.hadoop.hdfs.DistributedFileSystem$26.doCall(DistributedFileSystem.java:1442) at org.apache.hadoop.hdfs.DistributedFileSystem$26.doCall(DistributedFileSystem.java:1438) at org.apache.hadoop.fs.FileSystemLinkResolver.resolve(FileSystemLinkResolver.java:81) at org.apache.hadoop.hdfs.DistributedFileSystem.getFileStatus(DistributedFileSystem.java:1438) at org.apache.spark.deploy.yarn.ApplicationMaster$$anonfun$6.apply(ApplicationMaster.scala:160) at org.apache.spark.deploy.yarn.ApplicationMaster$$anonfun$6.apply(ApplicationMaster.scala:157) at scala.Option.foreach(Option.scala:257) at org.apache.spark.deploy.yarn.ApplicationMaster.<init>(ApplicationMaster.scala:157) at org.apache.spark.deploy.yarn.ApplicationMaster$$anonfun$main$1.apply$mcV$sp(ApplicationMaster.scala:765) at org.apache.spark.deploy.SparkHadoopUtil$$anon$1.run(SparkHadoopUtil.scala:67) at org.apache.spark.deploy.SparkHadoopUtil$$anon$1.run(SparkHadoopUtil.scala:66) at java.security.AccessController.doPrivileged(Native Method) at javax.security.auth.Subject.doAs(Subject.java:422) at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1866) at org.apache.spark.deploy.SparkHadoopUtil.runAsSparkUser(SparkHadoopUtil.scala:66) at org.apache.spark.deploy.yarn.ApplicationMaster$.main(ApplicationMaster.scala:764) at org.apache.spark.deploy.yarn.ApplicationMaster.main(ApplicationMaster.scala) On Fri, Jan 4, 2019 at 5:25 AM Marcelo Vanzin <van...@cloudera.com> wrote: > Ah, man, there are a few known issues with KMS delegation tokens. The main > one we've run into is HADOOP-14445, but it's only fixed in new versions of > Hadoop. I wouldn't expect you guys to be running those, but if you are, it > would be good to know. > > In our forks we added a hack to work around that issue, maybe you can try > it out: > > https://github.com/cloudera/spark/commit/108c1312d3a2b52090cb2713e7f8d68b9a0be8b1#diff-585a75e78c688c892d640281cfc56fed > > > On Thu, Jan 3, 2019 at 10:12 AM Paolo Platter <paolo.plat...@agilelab.it> > wrote: > >> Hi, >> >> >> >> The spark default behaviour is to request a brand new token every 24 >> hours, it is not going to renew delegation tokens, and it is the better >> approach for long running applications like streaming ones. >> >> >> >> In our use case using keytab and principal is working fine with >> hdfs_delegation_token but is NOT working with “kms-dt”. >> >> >> >> Anyone knows why this is happening ? Any suggestion to make it working >> with KMS ? >> >> >> >> Thanks >> >> >> >> >> >> >> >> [image: cid:image001.jpg@01D41D15.E01B6F00] >> >> *Paolo Platter* >> >> *CTO* >> >> E-mail: paolo.plat...@agilelab.it >> >> Web Site: www.agilelab.it >> >> >> >> >> ------------------------------ >> *Da:* Marcelo Vanzin <van...@cloudera.com.INVALID> >> *Inviato:* Thursday, January 3, 2019 7:03:22 PM >> *A:* alinazem...@gmail.com >> *Cc:* user >> *Oggetto:* Re: How to reissue a delegated token after max lifetime >> passes for a spark streaming application on a Kerberized cluster >> >> If you are using the principal / keytab params, Spark should create >> tokens as needed. If it's not, something else is going wrong, and only >> looking at full logs for the app would help. >> On Wed, Jan 2, 2019 at 5:09 PM Ali Nazemian <alinazem...@gmail.com> >> wrote: >> > >> > Hi, >> > >> > We are using a headless keytab to run our long-running spark streaming >> application. The token is renewed automatically every 1 day until it hits >> the max life limit. The problem is token is expired after max life (7 days) >> and we need to restart the job. Is there any way we can re-issue the token >> and pass it to a job that is already running? It doesn't feel right at all >> to restart the job every 7 days only due to the token issue. >> > >> > P.S: We use "--keytab /path/to/the/headless-keytab", "--principal >> principalNameAsPerTheKeytab" and "--conf >> spark.hadoop.fs.hdfs.impl.disable.cache=true" as the arguments for >> spark-submit command. >> > >> > Thanks, >> > Ali >> >> >> >> -- >> Marcelo >> >> --------------------------------------------------------------------- >> To unsubscribe e-mail: user-unsubscr...@spark.apache.org >> >> > > -- > Marcelo > -- A.Nazemian