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:27:11 UTC

[jira] [Created] (ACCUMULO-826) MapReduce over accumlo fails if process that started job is killed

Keith Turner created ACCUMULO-826:
-------------------------------------

             Summary: 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.0, 1.4.1
            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

[jira] [Updated] (ACCUMULO-826) MapReduce over accumlo fails if process that started job is killed

Posted by "Mike Drob (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/ACCUMULO-826?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Mike Drob updated ACCUMULO-826:
-------------------------------

    Affects Version/s:     (was: 1.4.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
>            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

[jira] [Commented] (ACCUMULO-826) MapReduce over accumlo fails if process that started job is killed

Posted by "Keith Turner (JIRA)" <ji...@apache.org>.
    [ 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

[jira] [Commented] (ACCUMULO-826) MapReduce over accumlo fails if process that started job is killed

Posted by "Christopher Tubbs (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/ACCUMULO-826?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13480436#comment-13480436 ] 

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

Thanks [~mdrob]! I hadn't realized it was only just fixed in 1.4.1. I propose we just revert to the 1.4.0 behavior of having base64 world-readable passwords in the config for 1.4.2, and fix it properly in 1.5.0 by accepting a Path to a file in HDFS that the user manages, to supply the password, as described in my previous comment.
                
> 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

[jira] [Comment Edited] (ACCUMULO-826) MapReduce over accumlo fails if process that started job is killed

Posted by "Keith Turner (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/ACCUMULO-826?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13480449#comment-13480449 ] 

Keith Turner edited comment on ACCUMULO-826 at 10/19/12 11:11 PM:
------------------------------------------------------------------

bq. I propose we just revert to the 1.4.0 behavior of having base64 world-readable passwords in the config for 1.4.2, and fix it properly in 1.5.0

I think this sounds reasonable.  The bug fix in 1.4.1 fixed and introduced bugs.  It slightly improved security, but not in any dramatic way (i.e. using cryptography).    Also, I can not find a way to delete the file when the job finishes as opposed to when the process that started the job exits.  Does anyone know of a way to delete the file when the job exits?  If we were to roll this back, should we roll back the changes to get and set ranges also?

bq.  using a shared secret (which I think Accumulo already has?)
Probably should not assume map reduce system can access accumulo's shared secret.  Christopher and I discussed using a public key to encrypt the password.  Would need to find a standard to use, its hard to get crypto protocols right.   But this is work for 1.5 I think, need to determine what we want to do for 1.4.2.
                
      was (Author: kturner):
    bq. I propose we just revert to the 1.4.0 behavior of having base64 world-readable passwords in the config for 1.4.2, and fix it properly in 1.5.0

I think this sounds reasonable.  The bug fix in 1.4.1 fixed and introduced bugs.  It slightly improved security, but not in any dramatic way (i.e. using cryptography).    Also, I can not find a way to delete the file when the job finishes as opposed to when the process that started the job exits.  Does anyone know of a way to delete the file when the job exits?  If we were to roll this back, should we roll back the changes to get and set ranges also?

bq.  using a shared secret (which I think Accumulo already has?)
Probably should not assume map reduce system can access accumulo's shared secret.  Christopher and I discussed using a public key to encrypt the password.  Would need to find a standard to use, its hard to get crypto protocols right.   But this is work for 1.5 I think.  
                  
> 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

[jira] [Resolved] (ACCUMULO-826) MapReduce over accumlo fails if process that started job is killed

Posted by "Keith Turner (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/ACCUMULO-826?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Keith Turner resolved ACCUMULO-826.
-----------------------------------

       Resolution: Fixed
    Fix Version/s: 1.4.2
         Assignee: Keith Turner

Rolled back changes in trunk also,  r1401004
                
> 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
>            Reporter: Keith Turner
>            Assignee: Keith Turner
>            Priority: Critical
>             Fix For: 1.4.2
>
>
> 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

[jira] [Commented] (ACCUMULO-826) MapReduce over accumlo fails if process that started job is killed

Posted by "Hudson (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/ACCUMULO-826?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13481726#comment-13481726 ] 

Hudson commented on ACCUMULO-826:
---------------------------------

Integrated in Accumulo-Trunk #533 (See [https://builds.apache.org/job/Accumulo-Trunk/533/])
    ACCUMULO-826 ACCUMULO-507 reverted changes that caused map reduce jobs to fail if the process that started the job exited (Revision 1401004)

     Result = SUCCESS
kturner : 
Files : 
* /accumulo/trunk
* /accumulo/trunk/assemble
* /accumulo/trunk/core
* /accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloOutputFormat.java
* /accumulo/trunk/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputFormatBase.java
* /accumulo/trunk/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloInputFormatTest.java
* /accumulo/trunk/fate/src/main/java/org/apache/accumulo/fate/ZooStore.java
* /accumulo/trunk/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooSession.java
* /accumulo/trunk/server
* /accumulo/trunk/src

                
> 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
>            Reporter: Keith Turner
>            Assignee: Keith Turner
>            Priority: Critical
>             Fix For: 1.4.2
>
>
> 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

[jira] [Commented] (ACCUMULO-826) MapReduce over accumlo fails if process that started job is killed

Posted by "Christopher Tubbs (JIRA)" <ji...@apache.org>.
    [ 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

[jira] [Commented] (ACCUMULO-826) MapReduce over accumlo fails if process that started job is killed

Posted by "Mike Drob (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/ACCUMULO-826?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13480407#comment-13480407 ] 

Mike Drob commented on ACCUMULO-826:
------------------------------------

The passwords are in files because of ACCUMULO-489.

I'm not a fan of adding more complexity to the client API by asking for a path to a file that contains the password instead of just asking for the password itself. Maybe it makes sense to still put the password in the job conf, but encrypt it using a shared secret (which I think Accumulo already has?). Oh, and make sure that the secret isn't in the conf as well.
                
> 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

[jira] [Commented] (ACCUMULO-826) MapReduce over accumlo fails if process that started job is killed

Posted by "Keith Turner (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/ACCUMULO-826?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13481568#comment-13481568 ] 

Keith Turner commented on ACCUMULO-826:
---------------------------------------

In revision 1400976 : reverted revisions 1397700,1382923,1339308,1339223,1336322.  These changes caused map reduce jobs to fail if the process that started the job exited.  

The reverse merge in svn worked very nicely.
                
> 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
>            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

[jira] [Commented] (ACCUMULO-826) MapReduce over accumlo fails if process that started job is killed

Posted by "Keith Turner (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/ACCUMULO-826?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13480449#comment-13480449 ] 

Keith Turner commented on ACCUMULO-826:
---------------------------------------

bq. I propose we just revert to the 1.4.0 behavior of having base64 world-readable passwords in the config for 1.4.2, and fix it properly in 1.5.0

I think this sounds reasonable.  The bug fix in 1.4.1 fixed and introduced bugs.  It slightly improved security, but not in any dramatic way (i.e. using cryptography).    Also, I can not find a way to delete the file when the job finishes as opposed to when the process that started the job exits.  Does anyone know of a way to delete the file when the job exits?  If we were to roll this back, should we roll back the changes to get and set ranges also?

bq.  using a shared secret (which I think Accumulo already has?)
Probably should not assume map reduce system can access accumulo's shared secret.  Christopher and I discussed using a public key to encrypt the password.  Would need to find a standard to use, its hard to get crypto protocols right.   But this is work for 1.5 I think.  
                
> 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

[jira] [Commented] (ACCUMULO-826) MapReduce over accumlo fails if process that started job is killed

Posted by "John Vines (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/ACCUMULO-826?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13481675#comment-13481675 ] 

John Vines commented on ACCUMULO-826:
-------------------------------------

The file gets stored in the private distributed cache, which was added in Hadoop 20.20something. The method for accessing that may not be accurate. Mike Drob is correct, that was implemented for ACCUMULO-489, which is a critical issue. The other implementation idea was having it stored temporarily in zookeeper. Having issues have to mess with the file system is worse, IMO. It will lead to users having passwords laying around in the filesystem world-readable because some do not know or do not care about securing their identity, they just want to run their MR job.

I think the only other secure implementation would be a token system implemented, but for the effort of timeliness, using the private distributed cache is a safe method of implementing this.
                
> 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
>            Reporter: Keith Turner
>            Assignee: Keith Turner
>            Priority: Critical
>             Fix For: 1.4.2
>
>
> 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

[jira] [Commented] (ACCUMULO-826) MapReduce over accumlo fails if process that started job is killed

Posted by "Hudson (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/ACCUMULO-826?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13481719#comment-13481719 ] 

Hudson commented on ACCUMULO-826:
---------------------------------

Integrated in Accumulo-1.4.x #244 (See [https://builds.apache.org/job/Accumulo-1.4.x/244/])
    ACCUMULO-826 ACCUMULO-507 reverted revisions 1397700,1382923,1339308,1339223,1336322.  These changes caused map reduce jobs to fail if the process that started the job exited. (Revision 1400976)

     Result = SUCCESS
kturner : 
Files : 
* /accumulo/branches/1.4/src/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloOutputFormat.java
* /accumulo/branches/1.4/src/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputFormatBase.java
* /accumulo/branches/1.4/src/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloInputFormatTest.java

                
> 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
>            Reporter: Keith Turner
>            Assignee: Keith Turner
>            Priority: Critical
>             Fix For: 1.4.2
>
>
> 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

[jira] [Commented] (ACCUMULO-826) MapReduce over accumlo fails if process that started job is killed

Posted by "Keith Turner (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/ACCUMULO-826?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13480360#comment-13480360 ] 

Keith Turner commented on ACCUMULO-826:
---------------------------------------

Seems like the methods to get and set ranges may have the same issues with delteOnExit and distributedCache.  Also, AccumuloOutputFormat.setOuputInfo() may have the same problem.
                
> 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