accumulo-notifications mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Christopher Tubbs (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (ACCUMULO-826) MapReduce over accumlo fails if process that started job is killed
Date Fri, 19 Oct 2012 20:28:13 GMT

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

Christopher Tubbs commented on ACCUMULO-826:
--------------------------------------------

I think a more important question is: why does the user's password get stored in a file without
their knowledge, as a side-effect of the implementation? I think this fundamentally goes against
the point of having static method to manipulate the job configuration (the point being to
only hold job state in the configuration, and not elsewhere with side-effects).

I do understand the benefit of not showing the user's password in the job configuration, which
is viewable from the JobTracker page, whereas the contents of this file wouldn't be. Perhaps
that was the reasoning for storing the password in a file in the first place. However, I think
it should be up to the user to manage this file's persistence, so we don't do unpredictable/unexpected
things, like create a file with their password in it without their knowledge or expectation,
or automatically delete the file when the client dies after the MapReduce job has been submitted,
therefore killing the MapReduce job.

I propose we modify the method signature to look like:
{code:java}
public static void setInputInfo(Configuration conf, String user, Path fileWithPasswordInHDFS,
String table, Authorizations auths);
{code}

Users could then re-use this file for multiple jobs, and they can control read/write access
to it.

This change may need to go through a deprecation path, and we may not want to do this until
1.5.0.
                
> 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

Mime
View raw message