[
https://issues.apache.org/jira/browse/YARN-4984?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16508220#comment-16508220
]
Bibin A Chundatt edited comment on YARN-4984 at 6/11/18 3:42 PM:
-----------------------------------------------------------------
[~leftnoteasy]/[~djp]
After the patch there could be a possible container log file leak for long
running containers.
Scenarios could be as follows.
# Submit application with runs for more than delegation token expiry time.
# Restart NM before expiry
# Start Nodemanager (On recovery the LogAggregationService start for app with
old token saved during container start)
# Invalid token exception will the thrown in createAppDir.
# Aggregators are removed on exception .--- change as per patch and exception
is thrown.
# Now rolling container aggregation thread / or stop of container logs will
not get aggregated.
IIUC earlier even if the AgregationInit fails once the heartbeat updates token
to NM .
{AppLogAggregatorImpl#uploadLogsForContainers} takes cares of adding the token
required for aggregation.
Please do correct me if i am wrong
was (Author: bibinchundatt):
[~leftnoteasy]/[~djp]
After the patch there could be a possible container log file leak for long
running containers.
Scenarios could be as follows.
# Submit application with runs for more than delegation token expiry time.
# Restart NM before expiry
# Start Nodemanager (On recovery the LogAggregationService start for app with
old token saved during submit)
# Invalid token will the thrown in createAppDir.
# Aggregators are removed on exception .
# Now rolling container aggregation thread/ on stop on container will not get
aggregated.
> LogAggregationService shouldn't swallow exception in handling createAppDir()
> which cause thread leak.
> -----------------------------------------------------------------------------------------------------
>
> Key: YARN-4984
> URL: https://issues.apache.org/jira/browse/YARN-4984
> Project: Hadoop YARN
> Issue Type: Bug
> Components: log-aggregation
> Affects Versions: 2.7.2
> Reporter: Junping Du
> Assignee: Junping Du
> Priority: Critical
> Fix For: 2.8.0, 3.0.0-alpha1
>
> Attachments: YARN-4984-v2.patch, YARN-4984-v3.patch,
> YARN-4984-v4.patch, YARN-4984.patch
>
>
> Due to YARN-4325, many stale applications still exists in NM state store and
> get recovered after NM restart. The app initiation will get failed due to
> token invalid, but exception is swallowed and aggregator thread is still
> created for invalid app.
> Exception is:
> {noformat}
> 158 2016-04-19 23:38:33,039 ERROR logaggregation.LogAggregationService
> (LogAggregationService.java:run(300)) - Failed to setup application log
> directory for application_1448 060878692_11842
> 159
> org.apache.hadoop.ipc.RemoteException(org.apache.hadoop.security.token.SecretManager$InvalidToken):
> token (HDFS_DELEGATION_TOKEN token 1380589 for hdfswrite) can't be fo
> und in cache
> 160 at org.apache.hadoop.ipc.Client.call(Client.java:1427)
> 161 at org.apache.hadoop.ipc.Client.call(Client.java:1358)
> 162 at
> org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:229)
> 163 at com.sun.proxy.$Proxy13.getFileInfo(Unknown Source)
> 164 at
> org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolTranslatorPB.getFileInfo(ClientNamenodeProtocolTranslatorPB.java:771)
> 165 at sun.reflect.GeneratedMethodAccessor76.invoke(Unknown
> Source)
> 166 at
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
> 167 at java.lang.reflect.Method.invoke(Method.java:606)
> 168 at
> org.apache.hadoop.io.retry.RetryInvocationHandler.invokeMethod(RetryInvocationHandler.java:252)
> 169 at
> org.apache.hadoop.io.retry.RetryInvocationHandler.invoke(RetryInvocationHandler.java:104)
> 170 at com.sun.proxy.$Proxy14.getFileInfo(Unknown Source)
> 171 at
> org.apache.hadoop.hdfs.DFSClient.getFileInfo(DFSClient.java:2116)
> 172 at
> org.apache.hadoop.hdfs.DistributedFileSystem$22.doCall(DistributedFileSystem.java:1315)
> 173 at
> org.apache.hadoop.hdfs.DistributedFileSystem$22.doCall(DistributedFileSystem.java:1311)
> 174 at
> org.apache.hadoop.fs.FileSystemLinkResolver.resolve(FileSystemLinkResolver.java:81)
> 175 at
> org.apache.hadoop.hdfs.DistributedFileSystem.getFileStatus(DistributedFileSystem.java:1311)
> 176 at
> org.apache.hadoop.yarn.server.nodemanager.containermanager.logaggregation.LogAggregationService.checkExists(LogAggregationService.java:248)
> 177 at
> org.apache.hadoop.yarn.server.nodemanager.containermanager.logaggregation.LogAggregationService.access$100(LogAggregationService.java:67)
> 178 at
> org.apache.hadoop.yarn.server.nodemanager.containermanager.logaggregation.LogAggregationService$1.run(LogAggregationService.java:276)
> 179 at java.security.AccessController.doPrivileged(Native Method)
> 180 at javax.security.auth.Subject.doAs(Subject.java:415)
> 181 at
> org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1657)
> 182 at
> org.apache.hadoop.yarn.server.nodemanager.containermanager.logaggregation.LogAggregationService.createAppDir(LogAggregationService.java:261)
> 183 at
> org.apache.hadoop.yarn.server.nodemanager.containermanager.logaggregation.LogAggregationService.initAppAggregator(LogAggregationService.java:367)
> 184 at
> org.apache.hadoop.yarn.server.nodemanager.containermanager.logaggregation.LogAggregationService.initApp(LogAggregationService.java:320)
> 185 at
> org.apache.hadoop.yarn.server.nodemanager.containermanager.logaggregation.LogAggregationService.handle(LogAggregationService.java:447)
> 186 at
> org.apache.hadoop.yarn.server.nodemanager.containermanager.logaggregation.LogAggregationService.handle(LogAggregationService.java:67)
> {noformat}
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]