You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@accumulo.apache.org by "Keith Turner (JIRA)" <ji...@apache.org> on 2012/10/19 21:46:12 UTC
[jira] [Commented] (ACCUMULO-826) MapReduce over accumlo fails if
process that started job is killed
[ https://issues.apache.org/jira/browse/ACCUMULO-826?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13480310#comment-13480310 ]
Keith Turner commented on ACCUMULO-826:
---------------------------------------
Why does the code that reads and write the password to a file put it in the DistributedCache and then read it directly from HDFS? Seems like the code that reads the password should use DistributedCache.getLocalCacheFiles().
> MapReduce over accumlo fails if process that started job is killed
> ------------------------------------------------------------------
>
> Key: ACCUMULO-826
> URL: https://issues.apache.org/jira/browse/ACCUMULO-826
> Project: Accumulo
> Issue Type: Bug
> Affects Versions: 1.4.1, 1.4.0
> Reporter: Keith Turner
> Priority: Critical
>
> While testing the 1.4.2rc2 I started a continuous verify and killed the process that started the job. Normally you would expect the job to keep running when you do this. Howerver task started to fail. I was seeing errors like the following.
> {noformat}
> java.io.FileNotFoundException: File does not exist: /user/hadoop/ContinuousVerify_13506740685261350674068686.pw
> at org.apache.hadoop.hdfs.DFSClient$DFSInputStream.openInfo(DFSClient.java:1685)
> at org.apache.hadoop.hdfs.DFSClient$DFSInputStream.<init>(DFSClient.java:1676)
> at org.apache.hadoop.hdfs.DFSClient.open(DFSClient.java:479)
> at org.apache.hadoop.hdfs.DistributedFileSystem.open(DistributedFileSystem.java:187)
> at org.apache.hadoop.fs.FileSystem.open(FileSystem.java:418)
> at org.apache.accumulo.core.client.mapreduce.InputFormatBase.getPassword(InputFormatBase.java:681)
> at org.apache.accumulo.core.client.mapreduce.InputFormatBase$RecordReaderBase.initialize(InputFormatBase.java:1155)
> at org.apache.hadoop.mapred.MapTask$NewTrackingRecordReader.initialize(MapTask.java:522)
> at org.apache.hadoop.mapred.MapTask.runNewMapper(MapTask.java:763)
> at org.apache.hadoop.mapred.MapTask.run(MapTask.java:370)
> at org.apache.hadoop.mapred.Child$4.run(Child.java:255)
> at java.security.AccessController.doPrivileged(Native Method)
> at javax.security.auth.Subject.doAs(Subject.java:396)
> at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1059)
> at org.apache.hadoop.mapred.Child.main(Child.java:249)
> {noformat}
> I think this is caused by the following code in InputFormatBase
> {code:java}
> public static void setInputInfo(Configuration conf, String user, byte[] passwd, String table, Authorizations auths) {
> if (conf.getBoolean(INPUT_INFO_HAS_BEEN_SET, false))
> throw new IllegalStateException("Input info can only be set once per job");
> conf.setBoolean(INPUT_INFO_HAS_BEEN_SET, true);
>
> ArgumentChecker.notNull(user, passwd, table);
> conf.set(USERNAME, user);
> conf.set(TABLE_NAME, table);
> if (auths != null && !auths.isEmpty())
> conf.set(AUTHORIZATIONS, auths.serialize());
>
> try {
> FileSystem fs = FileSystem.get(conf);
> Path file = new Path(fs.getWorkingDirectory(), conf.get("mapred.job.name") + System.currentTimeMillis() + ".pw");
> conf.set(PASSWORD_PATH, file.toString());
> FSDataOutputStream fos = fs.create(file, false);
> fs.setPermission(file, new FsPermission(FsAction.ALL, FsAction.NONE, FsAction.NONE));
> fs.deleteOnExit(file); // <--- NOT 100% sure, but I think this is the culprit
> {code}
--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira