Uploaded image for project: 'Hadoop Common'
  1. Hadoop Common
  2. HADOOP-13786

Add S3Guard committer for zero-rename commits to S3 endpoints

    Details

    • Type: New Feature
    • Status: Patch Available
    • Priority: Major
    • Resolution: Unresolved
    • Affects Version/s: 3.0.0-beta1
    • Fix Version/s: None
    • Component/s: fs/s3
    • Labels:
      None
    • Target Version/s:

      Description

      A goal of this code is "support O(1) commits to S3 repositories in the presence of failures". Implement it, including whatever is needed to demonstrate the correctness of the algorithm. (that is, assuming that s3guard provides a consistent view of the presence/absence of blobs, show that we can commit directly).

      I consider ourselves free to expose the blobstore-ness of the s3 output streams (ie. not visible until the close()), if we need to use that to allow us to abort commit operations.

      1. HADOOP-13786-HADOOP-13345-001.patch
        126 kB
        Steve Loughran
      2. HADOOP-13786-HADOOP-13345-002.patch
        150 kB
        Steve Loughran
      3. HADOOP-13786-HADOOP-13345-003.patch
        170 kB
        Steve Loughran
      4. HADOOP-13786-HADOOP-13345-004.patch
        222 kB
        Steve Loughran
      5. HADOOP-13786-HADOOP-13345-005.patch
        225 kB
        Steve Loughran
      6. HADOOP-13786-HADOOP-13345-006.patch
        231 kB
        Steve Loughran
      7. HADOOP-13786-HADOOP-13345-006.patch
        231 kB
        Steve Loughran
      8. HADOOP-13786-HADOOP-13345-007.patch
        246 kB
        Steve Loughran
      9. s3committer-master.zip
        42 kB
        Ryan Blue
      10. HADOOP-13786-HADOOP-13345-009.patch
        454 kB
        Steve Loughran
      11. HADOOP-13786-HADOOP-13345-010.patch
        471 kB
        Steve Loughran
      12. HADOOP-13786-HADOOP-13345-011.patch
        471 kB
        Steve Loughran
      13. HADOOP-13786-HADOOP-13345-012.patch
        484 kB
        Steve Loughran
      14. HADOOP-13786-HADOOP-13345-013.patch
        487 kB
        Steve Loughran
      15. HADOOP-13786-HADOOP-13345-015.patch
        499 kB
        Steve Loughran
      16. HADOOP-13786-HADOOP-13345-016.patch
        501 kB
        Steve Loughran
      17. HADOOP-13786-HADOOP-13345-017.patch
        505 kB
        Steve Loughran
      18. HADOOP-13786-HADOOP-13345-018.patch
        512 kB
        Steve Loughran
      19. HADOOP-13786-HADOOP-13345-019.patch
        521 kB
        Steve Loughran
      20. HADOOP-13786-HADOOP-13345-020.patch
        549 kB
        Steve Loughran
      21. HADOOP-13786-HADOOP-13345-021.patch
        558 kB
        Steve Loughran
      22. HADOOP-13786-HADOOP-13345-022.patch
        564 kB
        Steve Loughran
      23. HADOOP-13786-HADOOP-13345-023.patch
        597 kB
        Steve Loughran
      24. HADOOP-13786-HADOOP-13345-024.patch
        604 kB
        Steve Loughran
      25. objectstore.pdf
        187 kB
        Steve Loughran
      26. HADOOP-13786-HADOOP-13345-025.patch
        624 kB
        Steve Loughran
      27. HADOOP-13786-HADOOP-13345-026.patch
        656 kB
        Steve Loughran
      28. HADOOP-13786-HADOOP-13345-027.patch
        653 kB
        Steve Loughran
      29. HADOOP-13786-HADOOP-13345-028.patch
        669 kB
        Steve Loughran
      30. HADOOP-13786-HADOOP-13345-028.patch
        669 kB
        Steve Loughran
      31. HADOOP-13786-HADOOP-13345-029.patch
        674 kB
        Steve Loughran
      32. HADOOP-13786-HADOOP-13345-030.patch
        690 kB
        Steve Loughran
      33. HADOOP-13786-HADOOP-13345-031.patch
        704 kB
        Steve Loughran
      34. HADOOP-13786-HADOOP-13345-032.patch
        733 kB
        Steve Loughran
      35. HADOOP-13786-HADOOP-13345-033.patch
        748 kB
        Steve Loughran
      36. HADOOP-13786-HADOOP-13345-035.patch
        771 kB
        Steve Loughran
      37. cloud-intergration-test-failure.log
        334 kB
        Ewan Higgs
      38. HADOOP-13786-036.patch
        778 kB
        Steve Loughran
      39. HADOOP-13786-037.patch
        851 kB
        Steve Loughran
      40. HADOOP-13786-038.patch
        854 kB
        Steve Loughran
      41. HADOOP-13786-039.patch
        929 kB
        Steve Loughran

        Issue Links

          Activity

          Hide
          stevel@apache.org Steve Loughran added a comment -

          One thing to consider: an atomic PUT-no-overwrite can be used for speculative commits of a single file; I'm not so confident that it can be used for any task writing more than one file: it's no longer a single atomic commit at the end of the task. There's also the little detail which the output committer code in mapred.Task assumes that work is not-committed until the final communication with the AM; we'll need to make sure that cleanup always takes place. Maybe the

          Show
          stevel@apache.org Steve Loughran added a comment - One thing to consider: an atomic PUT-no-overwrite can be used for speculative commits of a single file; I'm not so confident that it can be used for any task writing more than one file: it's no longer a single atomic commit at the end of the task. There's also the little detail which the output committer code in mapred.Task assumes that work is not-committed until the final communication with the AM; we'll need to make sure that cleanup always takes place. Maybe the
          Hide
          stevel@apache.org Steve Loughran added a comment -

          Looking at the atomic commit problem, I'm starting to worry that I may need some directory leasing, which would imply a lock field in the table. Just warning you there. The problem is that while a rename is in progress, you wan't to stop anything trying to PUT data anywhere under the two paths being worked on.

          Show
          stevel@apache.org Steve Loughran added a comment - Looking at the atomic commit problem, I'm starting to worry that I may need some directory leasing, which would imply a lock field in the table. Just warning you there. The problem is that while a rename is in progress, you wan't to stop anything trying to PUT data anywhere under the two paths being worked on.
          Hide
          stevel@apache.org Steve Loughran added a comment -

          For the curious, my WiP, in the s3guard branch, is here: https://github.com/steveloughran/hadoop/tree/s3guard/HADOOP-13786-committer

          I've been tweaking the mapreduce code to add ability to switch to a subclass of FileOutputCommitter in FileOutputFormat (And transitively, all children), by way of a factory. The S3a factory dynamically chooses the committer based on the destination FS. There's currently no difference in the codebase apart from logging of operation durations. This means that we can switch the committer behind all output formats to using the s3a one (and also allowing anyone else to do the same with their own FOF committer subclass).

          I can see that ParquetOutputFormat has its own committer, ParquetOutputCommitter, as used by ParquetOutputFormat and so, I believe spark's dataframe.parquet.write() pipeline; my IDE isn't highlighting much else.

          Now, could I get away with just modifying the base FOF committer?

          for exception reporting it could use FileSystem.rename(final Path src, final Path dst, final Rename... options); S3A to impl that with exceptions. That's the "transitory" method for use between FS and FileContext, which does raise exceptions; the default is non-atomic and eventually gets to rename(src, dest), except for HDFS which does a full implementation. But: what if the semantics of that rename() (not yet in the FS spec, AFAIK) are different from what callers expect in some of the corner cases? And so commits everywhere break? That would not be good. And it would remove the ability to use any private s3a/s3guard calls if we wanted some (e.g. get a lock on a directory), unless they went in to the standard FS APIs.

          Making the output factory pluggable would avoid such problems, though logging duration might be nice all round. That said, given the time to rename, its somewhat unimportant when using real filesystems.

          Show
          stevel@apache.org Steve Loughran added a comment - For the curious, my WiP, in the s3guard branch, is here: https://github.com/steveloughran/hadoop/tree/s3guard/HADOOP-13786-committer I've been tweaking the mapreduce code to add ability to switch to a subclass of FileOutputCommitter in FileOutputFormat (And transitively, all children), by way of a factory. The S3a factory dynamically chooses the committer based on the destination FS. There's currently no difference in the codebase apart from logging of operation durations. This means that we can switch the committer behind all output formats to using the s3a one (and also allowing anyone else to do the same with their own FOF committer subclass). I can see that ParquetOutputFormat has its own committer, ParquetOutputCommitter , as used by ParquetOutputFormat and so, I believe spark's dataframe.parquet.write() pipeline; my IDE isn't highlighting much else. Now, could I get away with just modifying the base FOF committer? for exception reporting it could use FileSystem.rename(final Path src, final Path dst, final Rename... options) ; S3A to impl that with exceptions. That's the "transitory" method for use between FS and FileContext, which does raise exceptions; the default is non-atomic and eventually gets to rename(src, dest) , except for HDFS which does a full implementation. But: what if the semantics of that rename() (not yet in the FS spec, AFAIK) are different from what callers expect in some of the corner cases? And so commits everywhere break? That would not be good. And it would remove the ability to use any private s3a/s3guard calls if we wanted some (e.g. get a lock on a directory), unless they went in to the standard FS APIs. Making the output factory pluggable would avoid such problems, though logging duration might be nice all round. That said, given the time to rename, its somewhat unimportant when using real filesystems.
          Hide
          stevel@apache.org Steve Loughran added a comment -

          Update: the way s3a does commits via mergeOrUpdate() isn't really that atomic. It's atomic per destination rename, but as it treewalks doing a merge, there are actual potentials for races/inconsistent outcomes. What s3guard is at least guarantee that the view of the FS during that walk is consistent across clients.

          I'm checking in/pushing up an update, where I've been slowly moving up to some s3a internals, more for performance than consistency.

          Examples:

          • switch to innerDelete(FileStatus) for the delete, adding a parameter there to allow the check for maybe creating a parent dir to be skipped. As we know there's about be a new child, skipping that avoids 1-3 HEAD calls and a PUT which will soon be deleted
          • switch to an implementation of FileSystem.rename/3. This raises exceptions and lets us choose overwrite policy (good), but it still does two getFileStatusCalls, one for the src and one for the dest. As we have the source and have just deleted the test, we don't need them again. Having an @Privat method which took the source and dest values would strip out another 2-6 HTTP requests. Admittedly, on a large commit the cost of his preamble is low —the COPY becomes the expense.
          Show
          stevel@apache.org Steve Loughran added a comment - Update: the way s3a does commits via mergeOrUpdate() isn't really that atomic. It's atomic per destination rename, but as it treewalks doing a merge, there are actual potentials for races/inconsistent outcomes. What s3guard is at least guarantee that the view of the FS during that walk is consistent across clients. I'm checking in/pushing up an update, where I've been slowly moving up to some s3a internals, more for performance than consistency. Examples: switch to innerDelete(FileStatus) for the delete, adding a parameter there to allow the check for maybe creating a parent dir to be skipped. As we know there's about be a new child, skipping that avoids 1-3 HEAD calls and a PUT which will soon be deleted switch to an implementation of FileSystem.rename/3 . This raises exceptions and lets us choose overwrite policy (good), but it still does two getFileStatusCalls, one for the src and one for the dest. As we have the source and have just deleted the test, we don't need them again. Having an @Privat method which took the source and dest values would strip out another 2-6 HTTP requests. Admittedly, on a large commit the cost of his preamble is low —the COPY becomes the expense.
          Hide
          stevel@apache.org Steve Loughran added a comment -

          SPARK-18512 highlights a consistency problem surfacing during the mergePaths treewalk

          Show
          stevel@apache.org Steve Loughran added a comment - SPARK-18512 highlights a consistency problem surfacing during the mergePaths treewalk
          Hide
          abridgett Adrian Bridgett added a comment -

          Stack trace as requested (spark 2.0.2, hadoop 2.7.3):

           16/12/05 10:33:15 WARN scheduler.TaskSetManager: Lost task 170.0 in stage 10.0 (TID 3390, 10.1.201.20): org.apache.spark.SparkException: Task failed while writing rows
               at org.apache.spark.sql.execution.datasources.DefaultWriterContainer.writeRows(WriterContainer.scala:261)
               at org.apache.spark.sql.execution.datasources.InsertIntoHadoopFsRelationCommand$$anonfun$run$1$$anonfun$apply$mcV$sp$1.apply(InsertIntoHadoopFsRelationCommand.scala:143)
               at org.apache.spark.sql.execution.datasources.InsertIntoHadoopFsRelationCommand$$anonfun$run$1$$anonfun$apply$mcV$sp$1.apply(InsertIntoHadoopFsRelationCommand.scala:143)
               at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:70)
               at org.apache.spark.scheduler.Task.run(Task.scala:86)
               at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:274)
               at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
               at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
               at java.lang.Thread.run(Thread.java:745)
           Caused by: java.lang.RuntimeException: Failed to commit task
               at org.apache.spark.sql.execution.datasources.DefaultWriterContainer.org$apache$spark$sql$execution$datasources$DefaultWriterContainer$$commitTask$1(WriterContainer.scala:275)
               at org.apache.spark.sql.execution.datasources.DefaultWriterContainer$$anonfun$writeRows$1.apply$mcV$sp(WriterContainer.scala:257)
               at org.apache.spark.sql.execution.datasources.DefaultWriterContainer$$anonfun$writeRows$1.apply(WriterContainer.scala:252)
               at org.apache.spark.sql.execution.datasources.DefaultWriterContainer$$anonfun$writeRows$1.apply(WriterContainer.scala:252)
               at org.apache.spark.util.Utils$.tryWithSafeFinallyAndFailureCallbacks(Utils.scala:1348)
               at org.apache.spark.sql.execution.datasources.DefaultWriterContainer.writeRows(WriterContainer.scala:258)
               ... 8 more
           Caused by: java.io.IOException: Failed to rename S3AFileStatus{path=s3a://foo-scratch/adrian/dgm/nycextra/final/3dp/_temporary/0/_temporary/attempt_201612051033_0010_m_000170_0/part-r-00170-74752ee1-86f6-44e7-a5df-1c7b1e6fde47.csv.gz; isDirectory=false; length=32644; replication=1; blocksize=33554432; modification_time=1480933995000; access_time=0; owner=; group=; permission=rw-rw-rw-; isSymlink=false} to s3a://foo-scratch/adrian/dgm/nycextra/final/3dp/part-r-00170-74752ee1-86f6-44e7-a5df-1c7b1e6fde47.csv.gz
               at org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter.mergePaths(FileOutputCommitter.java:371)
               at org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter.mergePaths(FileOutputCommitter.java:384)
               at org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter.commitTask(FileOutputCommitter.java:486)
               at org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter.commitTask(FileOutputCommitter.java:449)
               at org.apache.spark.mapred.SparkHadoopMapRedUtil$.performCommit$1(SparkHadoopMapRedUtil.scala:50)
               at org.apache.spark.mapred.SparkHadoopMapRedUtil$.commitTask(SparkHadoopMapRedUtil.scala:76)
               at org.apache.spark.sql.execution.datasources.BaseWriterContainer.commitTask(WriterContainer.scala:211)
               at org.apache.spark.sql.execution.datasources.DefaultWriterContainer.org$apache$spark$sql$execution$datasources$DefaultWriterContainer$$commitTask$1(WriterContainer.scala:270)
               ... 13 more
          
          Show
          abridgett Adrian Bridgett added a comment - Stack trace as requested (spark 2.0.2, hadoop 2.7.3): 16/12/05 10:33:15 WARN scheduler.TaskSetManager: Lost task 170.0 in stage 10.0 (TID 3390, 10.1.201.20): org.apache.spark.SparkException: Task failed while writing rows at org.apache.spark.sql.execution.datasources.DefaultWriterContainer.writeRows(WriterContainer.scala:261) at org.apache.spark.sql.execution.datasources.InsertIntoHadoopFsRelationCommand$$anonfun$run$1$$anonfun$apply$mcV$sp$1.apply(InsertIntoHadoopFsRelationCommand.scala:143) at org.apache.spark.sql.execution.datasources.InsertIntoHadoopFsRelationCommand$$anonfun$run$1$$anonfun$apply$mcV$sp$1.apply(InsertIntoHadoopFsRelationCommand.scala:143) at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:70) at org.apache.spark.scheduler.Task.run(Task.scala:86) at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:274) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) at java.lang.Thread.run(Thread.java:745) Caused by: java.lang.RuntimeException: Failed to commit task at org.apache.spark.sql.execution.datasources.DefaultWriterContainer.org$apache$spark$sql$execution$datasources$DefaultWriterContainer$$commitTask$1(WriterContainer.scala:275) at org.apache.spark.sql.execution.datasources.DefaultWriterContainer$$anonfun$writeRows$1.apply$mcV$sp(WriterContainer.scala:257) at org.apache.spark.sql.execution.datasources.DefaultWriterContainer$$anonfun$writeRows$1.apply(WriterContainer.scala:252) at org.apache.spark.sql.execution.datasources.DefaultWriterContainer$$anonfun$writeRows$1.apply(WriterContainer.scala:252) at org.apache.spark.util.Utils$.tryWithSafeFinallyAndFailureCallbacks(Utils.scala:1348) at org.apache.spark.sql.execution.datasources.DefaultWriterContainer.writeRows(WriterContainer.scala:258) ... 8 more Caused by: java.io.IOException: Failed to rename S3AFileStatus{path=s3a://foo-scratch/adrian/dgm/nycextra/final/3dp/_temporary/0/_temporary/attempt_201612051033_0010_m_000170_0/part-r-00170-74752ee1-86f6-44e7-a5df-1c7b1e6fde47.csv.gz; isDirectory=false; length=32644; replication=1; blocksize=33554432; modification_time=1480933995000; access_time=0; owner=; group=; permission=rw-rw-rw-; isSymlink=false} to s3a://foo-scratch/adrian/dgm/nycextra/final/3dp/part-r-00170-74752ee1-86f6-44e7-a5df-1c7b1e6fde47.csv.gz at org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter.mergePaths(FileOutputCommitter.java:371) at org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter.mergePaths(FileOutputCommitter.java:384) at org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter.commitTask(FileOutputCommitter.java:486) at org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter.commitTask(FileOutputCommitter.java:449) at org.apache.spark.mapred.SparkHadoopMapRedUtil$.performCommit$1(SparkHadoopMapRedUtil.scala:50) at org.apache.spark.mapred.SparkHadoopMapRedUtil$.commitTask(SparkHadoopMapRedUtil.scala:76) at org.apache.spark.sql.execution.datasources.BaseWriterContainer.commitTask(WriterContainer.scala:211) at org.apache.spark.sql.execution.datasources.DefaultWriterContainer.org$apache$spark$sql$execution$datasources$DefaultWriterContainer$$commitTask$1(WriterContainer.scala:270) ... 13 more
          Hide
          stevel@apache.org Steve Loughran added a comment -

          thanks. I Don't see an easy fix, but I like to collect all data about failing commits.

          Show
          stevel@apache.org Steve Loughran added a comment - thanks. I Don't see an easy fix, but I like to collect all data about failing commits.
          Hide
          stevel@apache.org Steve Loughran added a comment -

          A few of us (Thomas, Pieter, Mingliang, Aaron and Sean) had a quick conf call earlier this week, where Thomas and Pieter outlined their proposed algorithm for implementing zero-rename commits to any consistent S3 endpoint.

          I've written up my interpretation of the algorithm for review, looking though the Hadoop and Spark commit code to see what appears to be going on, though I do need to actually document the various algorithms better.

          Comments welcome, especially those containing proofs of correctness

          Show
          stevel@apache.org Steve Loughran added a comment - A few of us (Thomas, Pieter, Mingliang, Aaron and Sean) had a quick conf call earlier this week, where Thomas and Pieter outlined their proposed algorithm for implementing zero-rename commits to any consistent S3 endpoint. I've written up my interpretation of the algorithm for review, looking though the Hadoop and Spark commit code to see what appears to be going on, though I do need to actually document the various algorithms better. Comments welcome, especially those containing proofs of correctness
          Hide
          Thomas Demoor Thomas Demoor added a comment -

          Had my write-up ready but you beat me too actually submitting Steve Loughran

          Filed HADOOP-13912 so we can track this separately, but I think it should go on top on what you have already done in the current ticket.

          Show
          Thomas Demoor Thomas Demoor added a comment - Had my write-up ready but you beat me too actually submitting Steve Loughran Filed HADOOP-13912 so we can track this separately, but I think it should go on top on what you have already done in the current ticket.
          Hide
          stevel@apache.org Steve Loughran added a comment -

          Patch 001

          1. modified FileOutputFormat to allow the committer to be chosen (as MRv1 did). Here's actually a factory which can be defined, so that you can have a committer factory which chooses the committer based on destination FS.
          2. There's an S3 factory, and a committer.
          3. The committer has some special methods in the FS API allowing it to bypass a lot of the checks before operations which the full Hadoop FS API requires. Here we can assume that the committer knows that the destination isn't a directory, doesn't want a mock parent directory created after deleting a path, etc.
          4. ITestS3AOutputCommitter is a clone of TestFileOutputCommitter, reworked to be against S3. Note it could be used as a basis for testing commits to other filesystems; the basic one assumes local FS.

          Some of the new tests are failing; I haven't completely weaned to the new tests off file:// and into being able to simulate different failures of (a subclass of) s3.

          Show
          stevel@apache.org Steve Loughran added a comment - Patch 001 modified FileOutputFormat to allow the committer to be chosen (as MRv1 did). Here's actually a factory which can be defined, so that you can have a committer factory which chooses the committer based on destination FS. There's an S3 factory, and a committer. The committer has some special methods in the FS API allowing it to bypass a lot of the checks before operations which the full Hadoop FS API requires. Here we can assume that the committer knows that the destination isn't a directory, doesn't want a mock parent directory created after deleting a path, etc. ITestS3AOutputCommitter is a clone of TestFileOutputCommitter , reworked to be against S3. Note it could be used as a basis for testing commits to other filesystems; the basic one assumes local FS. Some of the new tests are failing; I haven't completely weaned to the new tests off file:// and into being able to simulate different failures of (a subclass of) s3.
          Hide
          stevel@apache.org Steve Loughran added a comment -

          no worries; I had to write everything down to understand it more, look at the issues and estimate costs. I need to really define what the different committers do too, to make sure that I understand what they are doing and where.

          Show
          stevel@apache.org Steve Loughran added a comment - no worries; I had to write everything down to understand it more, look at the issues and estimate costs. I need to really define what the different committers do too, to make sure that I understand what they are doing and where .
          Hide
          liuml07 Mingliang Liu added a comment -

          Great work, Steve Loughran! I'll (learn and) review this year. Thanks,

          Show
          liuml07 Mingliang Liu added a comment - Great work, Steve Loughran ! I'll (learn and) review this year. Thanks,
          Hide
          stevel@apache.org Steve Loughran added a comment -

          im offline for a bit, and this patch doesn't do the new work, it's a precursor.

          Show
          stevel@apache.org Steve Loughran added a comment - im offline for a bit, and this patch doesn't do the new work, it's a precursor.
          Hide
          stevel@apache.org Steve Loughran added a comment -

          Making this standalone, giving title but making clear it's for any consistent endpoint. And about to discard all work in patch 1 except the algorithm doc

          Show
          stevel@apache.org Steve Loughran added a comment - Making this standalone, giving title but making clear it's for any consistent endpoint. And about to discard all work in patch 1 except the algorithm doc
          Hide
          stevel@apache.org Steve Loughran added a comment -

          Multipart algorithm summarised by Thomas Demoor on HADOOP-13912

          Object stores do not have an efficient rename operation, which is used by the Hadoop FileOutputCommitter to atomically promote the "winning" attempt out of the multiple (speculative) attempts to the final path. These slow job commits are one of the main friction points when using object stores in Hadoop.There have been quite some attempts at resolving this: HADOOP-9565Link, Apache Spark DirectOutputCommitters, ... but they have proven not to be robust in face of adversity (network partitions, ...).

          The current ticket proposes to do the atomic commit by using the S3 Multipart API, which allows multiple concurrent uploads on the same objectname, each in its own "temporary space, identified by the UploadId which is returned as a response to InitiateMultipartUpload. Every attempt writes directly to the final outputPath. Data is uploaded using Put Part and as a response an ETag for the part is returned and stored. The CompleteMultipartUpload is postponed. Instead, we persist the UploadId (using a _temporary subdir or elsewhere) and the ETags. When a certain "job" wins CompleteMultipartUpload is called for each of its files using the proper list of Part ETags.

          Completing a MultipartUpload is a metadata only operation (internally in S3) and is thus orders of magnitude faster than the rename-based approach which moves all the data.

          Show
          stevel@apache.org Steve Loughran added a comment - Multipart algorithm summarised by Thomas Demoor on HADOOP-13912 Object stores do not have an efficient rename operation, which is used by the Hadoop FileOutputCommitter to atomically promote the "winning" attempt out of the multiple (speculative) attempts to the final path. These slow job commits are one of the main friction points when using object stores in Hadoop.There have been quite some attempts at resolving this: HADOOP-9565 Link, Apache Spark DirectOutputCommitters, ... but they have proven not to be robust in face of adversity (network partitions, ...). The current ticket proposes to do the atomic commit by using the S3 Multipart API, which allows multiple concurrent uploads on the same objectname, each in its own "temporary space, identified by the UploadId which is returned as a response to InitiateMultipartUpload. Every attempt writes directly to the final outputPath. Data is uploaded using Put Part and as a response an ETag for the part is returned and stored. The CompleteMultipartUpload is postponed. Instead, we persist the UploadId (using a _temporary subdir or elsewhere) and the ETags. When a certain "job" wins CompleteMultipartUpload is called for each of its files using the proper list of Part ETags. Completing a MultipartUpload is a metadata only operation (internally in S3) and is thus orders of magnitude faster than the rename-based approach which moves all the data.
          Hide
          stevel@apache.org Steve Loughran added a comment -

          Final committer will need to handle speculative commits to same destination path.

          1. individual executors will need to write to job-specific subdir of a .temp_multipart_put dir
          2. central job committer to only commit work of tasks known to have completed
          3. ...and for those that did not complete: rm the pending multipart blocks by reading etags and deleting the pending blocks
          4. ideally: track destination filenames and catch conflict with >1 task generating same dest file.
          Show
          stevel@apache.org Steve Loughran added a comment - Final committer will need to handle speculative commits to same destination path. individual executors will need to write to job-specific subdir of a .temp_multipart_put dir central job committer to only commit work of tasks known to have completed ...and for those that did not complete: rm the pending multipart blocks by reading etags and deleting the pending blocks ideally: track destination filenames and catch conflict with >1 task generating same dest file.
          Hide
          stevel@apache.org Steve Loughran added a comment -

          Patch 002

          This patch

          1. defines the mechanism for using multipart uploads for commits
          2. implements it
          3. adds tests
          4. has tests working, including a scale one.

          It has not wired this up to MRv1/v2 output committers & FileOutputFormat, though the changes have been made to the MRv2 code to make it possible to place the S3A committer in behind FileOutputFormat.

          What works, then? Dela

          1. create a file with a path /example/__pending/job1/task1/part-001.bin
          2. this will initiate an MPU to /example/part001.bin, to which all the output goes.
          3. when the output stream is closed, the file /example/__pending/job1/task1/part-001.bin.pending is created, which saves everything needed to commit the job
          4. later a FileCommitActions class can be created.
          5. FileCommitActions .commitAllPendingFilesInPath() will scan a dir for .pending entries, and commit them one by one, here commitAllPendingFilesInPath("/example/__pending/job1/task1/")
          6. ..which causes the file /example/part001.bin to come into existence.
          7. or, if you call abortAllPendingFilesInPath(...) the MPUs are read and aborted.

          Performance? < 1s to commit a single 128MB file over a long-haul link.

          Duration of time to commit s3a://hwdev-steve-frankfurt-new/tests3ascale/scale/commit/__pending/job_001/commit.bin.pending: 688,701,514 nS
          

          I think that's pretty good

          Show
          stevel@apache.org Steve Loughran added a comment - Patch 002 This patch defines the mechanism for using multipart uploads for commits implements it adds tests has tests working, including a scale one. It has not wired this up to MRv1/v2 output committers & FileOutputFormat, though the changes have been made to the MRv2 code to make it possible to place the S3A committer in behind FileOutputFormat. What works, then? Dela create a file with a path /example/__pending/job1/task1/part-001.bin this will initiate an MPU to /example/part001.bin , to which all the output goes. when the output stream is closed, the file /example/__pending/job1/task1/part-001.bin.pending is created, which saves everything needed to commit the job later a FileCommitActions class can be created. FileCommitActions .commitAllPendingFilesInPath() will scan a dir for .pending entries, and commit them one by one, here commitAllPendingFilesInPath("/example/__pending/job1/task1/") ..which causes the file /example/part001.bin to come into existence. or, if you call abortAllPendingFilesInPath(...) the MPUs are read and aborted. Performance? < 1s to commit a single 128MB file over a long-haul link. Duration of time to commit s3a: //hwdev-steve-frankfurt- new /tests3ascale/scale/commit/__pending/job_001/commit.bin.pending: 688,701,514 nS I think that's pretty good
          Hide
          stevel@apache.org Steve Loughran added a comment -

          MR test failure assumed spurious:

          java.util.ServiceConfigurationError: org.apache.hadoop.security.token.TokenIdentifier: Provider org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier not found
          	at java.util.ServiceLoader.fail(ServiceLoader.java:239)
          	at java.util.ServiceLoader.access$300(ServiceLoader.java:185)
          	at java.util.ServiceLoader$LazyIterator.nextService(ServiceLoader.java:372)
          	at java.util.ServiceLoader$LazyIterator.next(ServiceLoader.java:404)
          	at java.util.ServiceLoader$1.next(ServiceLoader.java:480)
          	at org.apache.hadoop.security.token.Token.getClassForIdentifier(Token.java:149)
          	at org.apache.hadoop.security.token.Token.decodeIdentifier(Token.java:170)
          	at org.apache.hadoop.security.token.Token.identifierToString(Token.java:425)
          	at org.apache.hadoop.security.token.Token.toString(Token.java:445)
          	at java.lang.String.valueOf(String.java:2994)
          	at java.lang.StringBuilder.append(StringBuilder.java:131)
          	at org.apache.hadoop.mapreduce.security.TokenCache.obtainTokensForNamenodesInternal(TokenCache.java:143)
          	at org.apache.hadoop.mapreduce.security.TestTokenCache.testBinaryCredentials(TestTokenCache.java:116)
          	at org.apache.hadoop.mapreduce.security.TestTokenCache.testBinaryCredentialsWithoutScheme(TestTokenCache.java:67)
          

          otherwise, there isn't enough test coverage, or even the MR integration, for this to be ready to think about committing. I've stuck it up to give people progress.

          Key point: we can use delayed MPU completion for 0-rename writes, caching the information needed for the operation in S3 itself. We are yet to show that this works for MR code

          Show
          stevel@apache.org Steve Loughran added a comment - MR test failure assumed spurious: java.util.ServiceConfigurationError: org.apache.hadoop.security.token.TokenIdentifier: Provider org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier not found at java.util.ServiceLoader.fail(ServiceLoader.java:239) at java.util.ServiceLoader.access$300(ServiceLoader.java:185) at java.util.ServiceLoader$LazyIterator.nextService(ServiceLoader.java:372) at java.util.ServiceLoader$LazyIterator.next(ServiceLoader.java:404) at java.util.ServiceLoader$1.next(ServiceLoader.java:480) at org.apache.hadoop.security.token.Token.getClassForIdentifier(Token.java:149) at org.apache.hadoop.security.token.Token.decodeIdentifier(Token.java:170) at org.apache.hadoop.security.token.Token.identifierToString(Token.java:425) at org.apache.hadoop.security.token.Token.toString(Token.java:445) at java.lang. String .valueOf( String .java:2994) at java.lang.StringBuilder.append(StringBuilder.java:131) at org.apache.hadoop.mapreduce.security.TokenCache.obtainTokensForNamenodesInternal(TokenCache.java:143) at org.apache.hadoop.mapreduce.security.TestTokenCache.testBinaryCredentials(TestTokenCache.java:116) at org.apache.hadoop.mapreduce.security.TestTokenCache.testBinaryCredentialsWithoutScheme(TestTokenCache.java:67) otherwise, there isn't enough test coverage, or even the MR integration, for this to be ready to think about committing. I've stuck it up to give people progress. Key point: we can use delayed MPU completion for 0-rename writes, caching the information needed for the operation in S3 itself. We are yet to show that this works for MR code
          Hide
          stevel@apache.org Steve Loughran added a comment -

          Patch 003. This

          • implements the s3a committer
          • has both the commitPath and abortPath succeed if the path doesn't exist. This avoids having to do an existence check before invocation.
          • adds tests for the abort
          • plays around with the policy as to "what do do on an abort if the file named isn't there". It now catches the exception and ignores it unless told otherwise.
          • has the pom setup for writing the MiniMRYarnCluster test.

          The next task is that test setup; once that is done we can try to generate different situations: parallel runs, failures, etc.

          Show
          stevel@apache.org Steve Loughran added a comment - Patch 003. This implements the s3a committer has both the commitPath and abortPath succeed if the path doesn't exist. This avoids having to do an existence check before invocation. adds tests for the abort plays around with the policy as to "what do do on an abort if the file named isn't there". It now catches the exception and ignores it unless told otherwise. has the pom setup for writing the MiniMRYarnCluster test. The next task is that test setup; once that is done we can try to generate different situations: parallel runs, failures, etc.
          Hide
          stevel@apache.org Steve Loughran added a comment -

          Patch 004

          this is passing the tests in a suite derived from org.apache.hadoop.mapreduce.lib.output.TestFileOutputCommitter;still looking at ways to simulate failure conditions and semantics of failure we want.

          Essentially: once a pending commit has happened, there is no retry. Meaning: when a task has committed once, it should fail from then on, which it does with an FNFE on the task attempt dir.

          Similarly you can only commit a job once, even if all the job does is delete all child directories.

          One change in this patch is the need to support pending subtrees, eg. map output to the directory part-0000/index and part-0000/data in the destination dir; this has been done by adding the notion of a _base path element in the pending tree. When a base path is a parent. the destination path is the parent of the __pending dir, with all children under base retained. With each task attempt dir being dest/pending/$app/$app-attempt/$task_attempt/_base, this ensures that all data created in the task working dir ends up under the destination in the same directory tree.

          issues:

          • what about cleaning up __pending? Job commit?
          • need to stop someone creating a path _base/_pending and so sneak in pending stuff/get very confused. Actually, stop __pending under __pending.
          Show
          stevel@apache.org Steve Loughran added a comment - Patch 004 this is passing the tests in a suite derived from org.apache.hadoop.mapreduce.lib.output.TestFileOutputCommitter ;still looking at ways to simulate failure conditions and semantics of failure we want. Essentially: once a pending commit has happened, there is no retry . Meaning: when a task has committed once, it should fail from then on, which it does with an FNFE on the task attempt dir. Similarly you can only commit a job once, even if all the job does is delete all child directories. One change in this patch is the need to support pending subtrees, eg. map output to the directory part-0000/index and part-0000/data in the destination dir; this has been done by adding the notion of a _ base path element in the pending tree. When a base path is a parent. the destination path is the parent of the __pending dir, with all children under base retained. With each task attempt dir being dest/ pending/$app/$app-attempt/$task_attempt/ _base , this ensures that all data created in the task working dir ends up under the destination in the same directory tree. issues: what about cleaning up __pending? Job commit? need to stop someone creating a path _ base/ _pending and so sneak in pending stuff/get very confused. Actually, stop __pending under __pending.
          Hide
          stevel@apache.org Steve Loughran added a comment -

          About to add a new patch, which does more in terms of testing, though there's some ambiguity about the semantics of commit and abort that I need to clarify with the MR Team.

          This code works without s3guard, but fails (differently) with s3guard local and s3guard dynamo

          s3guard DDDB

          Running org.apache.hadoop.fs.s3a.commit.ITestS3AOutputCommitter
          Tests run: 9, Failures: 2, Errors: 0, Skipped: 0, Time elapsed: 94.915 sec <<< FAILURE! - in org.apache.hadoop.fs.s3a.commit.ITestS3AOutputCommitter
          testAbort(org.apache.hadoop.fs.s3a.commit.ITestS3AOutputCommitter)  Time elapsed: 9.292 sec  <<< FAILURE!
          java.lang.AssertionError: Output directory not empty ls s3a://hwdev-steve-ireland-new/test/testAbort [00] S3AFileStatus{path=s3a://hwdev-steve-ireland-new/test/testAbort/part-m-00000; isDirectory=false; length=40; replication=1; blocksize=33554432; modification_time=1486142401246; access_time=0; owner=stevel; group=stevel; permission=rw-rw-rw-; isSymlink=false} isEmptyDirectory=false
          : array lengths differed, expected.length=0 actual.length=1
                  at org.junit.Assert.fail(Assert.java:88)
                  at org.junit.internal.ComparisonCriteria.assertArraysAreSameLength(ComparisonCriteria.java:71)
                  at org.junit.internal.ComparisonCriteria.arrayEquals(ComparisonCriteria.java:32)
                  at org.junit.Assert.internalArrayEquals(Assert.java:473)
                  at org.junit.Assert.assertArrayEquals(Assert.java:265)
                  at org.apache.hadoop.fs.s3a.commit.ITestS3AOutputCommitter.testAbort(ITestS3AOutputCommitter.java:561)
          
          testFailAbort(org.apache.hadoop.fs.s3a.commit.ITestS3AOutputCommitter)  Time elapsed: 9.453 sec  <<< FAILURE!
          java.lang.AssertionError: expected output file: unexpectedly found s3a://hwdev-steve-ireland-new/test/testFailAbort/part-m-00000 as  S3AFileStatus{path=s3a://hwdev-steve-ireland-new/test/testFailAbort/part-m-00000; isDirectory=false; length=40; replication=1; blocksize=33554432; modification_time=1486142441390; access_time=0; owner=stevel; group=stevel; permission=rw-rw-rw-; isSymlink=false} isEmptyDirectory=false
                  at org.junit.Assert.fail(Assert.java:88)
                  at org.apache.hadoop.fs.contract.ContractTestUtils.assertPathDoesNotExist(ContractTestUtils.java:796)
                  at org.apache.hadoop.fs.contract.AbstractFSContractTestBase.assertPathDoesNotExist(AbstractFSContractTestBase.java:305)
                  at org.apache.hadoop.fs.s3a.commit.ITestS3AOutputCommitter.testFailAbort(ITestS3AOutputCommitter.java:587)
          
          
          Results :
          
          Failed tests: 
            ITestS3AOutputCommitter.testAbort:561->Assert.assertArrayEquals:265->Assert.internalArrayEquals:473->Assert.fail:88 Output directory not empty ls s3a://hwdev-steve-ireland-new/test/testAbort [00] S3AFileStatus{path=s3a://hwdev-steve-ireland-new/test/testAbort/part-m-00000; isDirectory=false; length=40; replication=1; blocksize=33554432; modification_time=1486142401246; access_time=0; owner=stevel; group=stevel; permission=rw-rw-rw-; isSymlink=false} isEmptyDirectory=false
          : array lengths differed, expected.length=0 actual.length=1
            ITestS3AOutputCommitter.testFailAbort:587->AbstractFSContractTestBase.assertPathDoesNotExist:305->Assert.fail:88 expected output file: unexpectedly found s3a://hwdev-steve-ireland-new/test/testFailAbort/part-m-00000 as  S3AFileStatus{path=s3a://hwdev-steve-ireland-new/test/testFailAbort/part-m-00000; isDirectory=false; length=40; replication=1; blocksize=33554432; modification_time=1486142441390; access_time=0; owner=stevel; group=stevel; permission=rw-rw-rw-; isSymlink=false} isEmptyDirectory=false
          
          Tests run: 9, Failures: 2, Errors: 0, Skipped: 0
          
          
          

          s3guard local DB

          -------------------------------------------------------
           T E S T S
          -------------------------------------------------------
          Running org.apache.hadoop.fs.s3a.commit.ITestS3AOutputCommitter
          Tests run: 9, Failures: 2, Errors: 2, Skipped: 0, Time elapsed: 53.226 sec <<< FAILURE! - in org.apache.hadoop.fs.s3a.commit.ITestS3AOutputCommitter
          testMapFileOutputCommitter(org.apache.hadoop.fs.s3a.commit.ITestS3AOutputCommitter)  Time elapsed: 9.394 sec  <<< FAILURE!
          java.lang.AssertionError: Number of MapFile.Reader entries in s3a://hwdev-steve-ireland-new/test/testMapFileOutputCommitter : ls s3a://hwdev-steve-ireland-new/test/testMapFileOutputCommitter [00] S3AFileStatus{path=s3a://hwdev-steve-ireland-new/test/testMapFileOutputCommitter/_SUCCESS; isDirectory=false; length=0; replication=1; blocksize=33554432; modification_time=1486142538000; access_time=0; owner=stevel; group=stevel; permission=rw-rw-rw-; isSymlink=false} isEmptyDirectory=false
           expected:<1> but was:<0>
                  at org.junit.Assert.fail(Assert.java:88)
                  at org.junit.Assert.failNotEquals(Assert.java:743)
                  at org.junit.Assert.assertEquals(Assert.java:118)
                  at org.junit.Assert.assertEquals(Assert.java:555)
                  at org.apache.hadoop.fs.s3a.commit.ITestS3AOutputCommitter.testMapFileOutputCommitter(ITestS3AOutputCommitter.java:500)
          
          testAbort(org.apache.hadoop.fs.s3a.commit.ITestS3AOutputCommitter)  Time elapsed: 4.307 sec  <<< ERROR!
          java.io.FileNotFoundException: No such file or directory: s3a://hwdev-steve-ireland-new/test/testAbort
                  at org.apache.hadoop.fs.s3a.S3AFileSystem.getFileStatus(S3AFileSystem.java:1765)
                  at org.apache.hadoop.fs.s3a.S3AFileSystem.innerListStatus(S3AFileSystem.java:1480)
                  at org.apache.hadoop.fs.s3a.S3AFileSystem.listStatus(S3AFileSystem.java:1456)
                  at org.apache.hadoop.fs.contract.ContractTestUtils.listChildren(ContractTestUtils.java:427)
                  at org.apache.hadoop.fs.s3a.commit.ITestS3AOutputCommitter.testAbort(ITestS3AOutputCommitter.java:560)
          
          testCommitterWithFailure(org.apache.hadoop.fs.s3a.commit.ITestS3AOutputCommitter)  Time elapsed: 5.375 sec  <<< FAILURE!
          java.lang.AssertionError: Expected an exception
                  at org.apache.hadoop.test.LambdaTestUtils.intercept(LambdaTestUtils.java:374)
                  at org.apache.hadoop.fs.s3a.commit.ITestS3AOutputCommitter.expectFNFEonJobCommit(ITestS3AOutputCommitter.java:396)
                  at org.apache.hadoop.fs.s3a.commit.ITestS3AOutputCommitter.testCommitterWithFailure(ITestS3AOutputCommitter.java:390)
          
          testFailAbort(org.apache.hadoop.fs.s3a.commit.ITestS3AOutputCommitter)  Time elapsed: 5.016 sec  <<< ERROR!
          java.io.FileNotFoundException: expected output dir: not found s3a://hwdev-steve-ireland-new/test/testFailAbort in s3a://hwdev-steve-ireland-new/test
                  at org.apache.hadoop.fs.s3a.S3AFileSystem.getFileStatus(S3AFileSystem.java:1765)
                  at org.apache.hadoop.fs.s3a.S3AFileSystem.getFileStatus(S3AFileSystem.java:125)
                  at org.apache.hadoop.fs.contract.ContractTestUtils.verifyPathExists(ContractTestUtils.java:773)
                  at org.apache.hadoop.fs.contract.ContractTestUtils.assertPathExists(ContractTestUtils.java:757)
                  at org.apache.hadoop.fs.contract.AbstractFSContractTestBase.assertPathExists(AbstractFSContractTestBase.java:294)
                  at org.apache.hadoop.fs.s3a.commit.ITestS3AOutputCommitter.testFailAbort(ITestS3AOutputCommitter.java:586)
          
          

          I don't know what's up here, but I do know that (a) we're doing too many deletes during the commit process, more specifically: creating too many mock empty dirs. That can be optimised with a "delete don't care about a parent dir" method in writeOperationHelper.

          In the first s3guard local test, testMapFileOutputCommitter all is well in the FS used by the job, but when a new FS instance is created in the same process, the second one isn't seeing the listing.

          Not looked at the other failures in any detail.

          Show
          stevel@apache.org Steve Loughran added a comment - About to add a new patch, which does more in terms of testing, though there's some ambiguity about the semantics of commit and abort that I need to clarify with the MR Team. This code works without s3guard, but fails (differently) with s3guard local and s3guard dynamo s3guard DDDB Running org.apache.hadoop.fs.s3a.commit.ITestS3AOutputCommitter Tests run: 9, Failures: 2, Errors: 0, Skipped: 0, Time elapsed: 94.915 sec <<< FAILURE! - in org.apache.hadoop.fs.s3a.commit.ITestS3AOutputCommitter testAbort(org.apache.hadoop.fs.s3a.commit.ITestS3AOutputCommitter) Time elapsed: 9.292 sec <<< FAILURE! java.lang.AssertionError: Output directory not empty ls s3a: //hwdev-steve-ireland- new /test/testAbort [00] S3AFileStatus{path=s3a://hwdev-steve-ireland- new /test/testAbort/part-m-00000; isDirectory= false ; length=40; replication=1; blocksize=33554432; modification_time=1486142401246; access_time=0; owner=stevel; group=stevel; permission=rw-rw-rw-; isSymlink= false } isEmptyDirectory= false : array lengths differed, expected.length=0 actual.length=1 at org.junit.Assert.fail(Assert.java:88) at org.junit.internal.ComparisonCriteria.assertArraysAreSameLength(ComparisonCriteria.java:71) at org.junit.internal.ComparisonCriteria.arrayEquals(ComparisonCriteria.java:32) at org.junit.Assert.internalArrayEquals(Assert.java:473) at org.junit.Assert.assertArrayEquals(Assert.java:265) at org.apache.hadoop.fs.s3a.commit.ITestS3AOutputCommitter.testAbort(ITestS3AOutputCommitter.java:561) testFailAbort(org.apache.hadoop.fs.s3a.commit.ITestS3AOutputCommitter) Time elapsed: 9.453 sec <<< FAILURE! java.lang.AssertionError: expected output file: unexpectedly found s3a: //hwdev-steve-ireland- new /test/testFailAbort/part-m-00000 as S3AFileStatus{path=s3a://hwdev-steve-ireland- new /test/testFailAbort/part-m-00000; isDirectory= false ; length=40; replication=1; blocksize=33554432; modification_time=1486142441390; access_time=0; owner=stevel; group=stevel; permission=rw-rw-rw-; isSymlink= false } isEmptyDirectory= false at org.junit.Assert.fail(Assert.java:88) at org.apache.hadoop.fs.contract.ContractTestUtils.assertPathDoesNotExist(ContractTestUtils.java:796) at org.apache.hadoop.fs.contract.AbstractFSContractTestBase.assertPathDoesNotExist(AbstractFSContractTestBase.java:305) at org.apache.hadoop.fs.s3a.commit.ITestS3AOutputCommitter.testFailAbort(ITestS3AOutputCommitter.java:587) Results : Failed tests: ITestS3AOutputCommitter.testAbort:561->Assert.assertArrayEquals:265->Assert.internalArrayEquals:473->Assert.fail:88 Output directory not empty ls s3a: //hwdev-steve-ireland- new /test/testAbort [00] S3AFileStatus{path=s3a://hwdev-steve-ireland- new /test/testAbort/part-m-00000; isDirectory= false ; length=40; replication=1; blocksize=33554432; modification_time=1486142401246; access_time=0; owner=stevel; group=stevel; permission=rw-rw-rw-; isSymlink= false } isEmptyDirectory= false : array lengths differed, expected.length=0 actual.length=1 ITestS3AOutputCommitter.testFailAbort:587->AbstractFSContractTestBase.assertPathDoesNotExist:305->Assert.fail:88 expected output file: unexpectedly found s3a: //hwdev-steve-ireland- new /test/testFailAbort/part-m-00000 as S3AFileStatus{path=s3a://hwdev-steve-ireland- new /test/testFailAbort/part-m-00000; isDirectory= false ; length=40; replication=1; blocksize=33554432; modification_time=1486142441390; access_time=0; owner=stevel; group=stevel; permission=rw-rw-rw-; isSymlink= false } isEmptyDirectory= false Tests run: 9, Failures: 2, Errors: 0, Skipped: 0 s3guard local DB ------------------------------------------------------- T E S T S ------------------------------------------------------- Running org.apache.hadoop.fs.s3a.commit.ITestS3AOutputCommitter Tests run: 9, Failures: 2, Errors: 2, Skipped: 0, Time elapsed: 53.226 sec <<< FAILURE! - in org.apache.hadoop.fs.s3a.commit.ITestS3AOutputCommitter testMapFileOutputCommitter(org.apache.hadoop.fs.s3a.commit.ITestS3AOutputCommitter) Time elapsed: 9.394 sec <<< FAILURE! java.lang.AssertionError: Number of MapFile.Reader entries in s3a: //hwdev-steve-ireland- new /test/testMapFileOutputCommitter : ls s3a://hwdev-steve-ireland- new /test/testMapFileOutputCommitter [00] S3AFileStatus{path=s3a://hwdev-steve-ireland- new /test/testMapFileOutputCommitter/_SUCCESS; isDirectory= false ; length=0; replication=1; blocksize=33554432; modification_time=1486142538000; access_time=0; owner=stevel; group=stevel; permission=rw-rw-rw-; isSymlink= false } isEmptyDirectory= false expected:<1> but was:<0> at org.junit.Assert.fail(Assert.java:88) at org.junit.Assert.failNotEquals(Assert.java:743) at org.junit.Assert.assertEquals(Assert.java:118) at org.junit.Assert.assertEquals(Assert.java:555) at org.apache.hadoop.fs.s3a.commit.ITestS3AOutputCommitter.testMapFileOutputCommitter(ITestS3AOutputCommitter.java:500) testAbort(org.apache.hadoop.fs.s3a.commit.ITestS3AOutputCommitter) Time elapsed: 4.307 sec <<< ERROR! java.io.FileNotFoundException: No such file or directory: s3a: //hwdev-steve-ireland- new /test/testAbort at org.apache.hadoop.fs.s3a.S3AFileSystem.getFileStatus(S3AFileSystem.java:1765) at org.apache.hadoop.fs.s3a.S3AFileSystem.innerListStatus(S3AFileSystem.java:1480) at org.apache.hadoop.fs.s3a.S3AFileSystem.listStatus(S3AFileSystem.java:1456) at org.apache.hadoop.fs.contract.ContractTestUtils.listChildren(ContractTestUtils.java:427) at org.apache.hadoop.fs.s3a.commit.ITestS3AOutputCommitter.testAbort(ITestS3AOutputCommitter.java:560) testCommitterWithFailure(org.apache.hadoop.fs.s3a.commit.ITestS3AOutputCommitter) Time elapsed: 5.375 sec <<< FAILURE! java.lang.AssertionError: Expected an exception at org.apache.hadoop.test.LambdaTestUtils.intercept(LambdaTestUtils.java:374) at org.apache.hadoop.fs.s3a.commit.ITestS3AOutputCommitter.expectFNFEonJobCommit(ITestS3AOutputCommitter.java:396) at org.apache.hadoop.fs.s3a.commit.ITestS3AOutputCommitter.testCommitterWithFailure(ITestS3AOutputCommitter.java:390) testFailAbort(org.apache.hadoop.fs.s3a.commit.ITestS3AOutputCommitter) Time elapsed: 5.016 sec <<< ERROR! java.io.FileNotFoundException: expected output dir: not found s3a: //hwdev-steve-ireland- new /test/testFailAbort in s3a://hwdev-steve-ireland- new /test at org.apache.hadoop.fs.s3a.S3AFileSystem.getFileStatus(S3AFileSystem.java:1765) at org.apache.hadoop.fs.s3a.S3AFileSystem.getFileStatus(S3AFileSystem.java:125) at org.apache.hadoop.fs.contract.ContractTestUtils.verifyPathExists(ContractTestUtils.java:773) at org.apache.hadoop.fs.contract.ContractTestUtils.assertPathExists(ContractTestUtils.java:757) at org.apache.hadoop.fs.contract.AbstractFSContractTestBase.assertPathExists(AbstractFSContractTestBase.java:294) at org.apache.hadoop.fs.s3a.commit.ITestS3AOutputCommitter.testFailAbort(ITestS3AOutputCommitter.java:586) I don't know what's up here, but I do know that (a) we're doing too many deletes during the commit process, more specifically: creating too many mock empty dirs. That can be optimised with a "delete don't care about a parent dir" method in writeOperationHelper. In the first s3guard local test, testMapFileOutputCommitter all is well in the FS used by the job, but when a new FS instance is created in the same process, the second one isn't seeing the listing. Not looked at the other failures in any detail.
          Hide
          stevel@apache.org Steve Loughran added a comment -

          Patch 006

          1. make sure that after a multipart put is completed, the metastore is told
          2. a bit more diagnostics on why a test is only failing on s3guard local
          3. added writehelper allows caller to delete a file and not ask for any parent dir to be optionally created. This will allow for a faster scan & commit of all files in a directory tree, after which the tree gets deleted anyway. This is not yet activated.
          4. Fixes findbugs reports (some were real!); javac, javadoc, many of the checkstyle.
          Show
          stevel@apache.org Steve Loughran added a comment - Patch 006 make sure that after a multipart put is completed, the metastore is told a bit more diagnostics on why a test is only failing on s3guard local added writehelper allows caller to delete a file and not ask for any parent dir to be optionally created. This will allow for a faster scan & commit of all files in a directory tree, after which the tree gets deleted anyway. This is not yet activated. Fixes findbugs reports (some were real!); javac, javadoc, many of the checkstyle.
          Hide
          stevel@apache.org Steve Loughran added a comment -

          tested s3a ireland, one failure

          Failed tests: 
            ITestS3ACommitOperations.testCreateAbortEmptyFile:127->Assert.assertFalse:64->Assert.assertTrue:41->Assert.fail:88 Expected 2nd abort to fail CommitFileOutcome{success, destination=null, pendingFile=s3a://hwdev-steve-ireland-new/test/testCreateAbortEmptyFile/__pending/empty-abort.txt.pending}
          
          
          Show
          stevel@apache.org Steve Loughran added a comment - tested s3a ireland, one failure Failed tests: ITestS3ACommitOperations.testCreateAbortEmptyFile:127->Assert.assertFalse:64->Assert.assertTrue:41->Assert.fail:88 Expected 2nd abort to fail CommitFileOutcome{success, destination= null , pendingFile=s3a: //hwdev-steve-ireland- new /test/testCreateAbortEmptyFile/__pending/empty-abort.txt.pending}
          Hide
          stevel@apache.org Steve Loughran added a comment -

          I've realised we can also do delayed-commit object COPY too, because multipart PUT lets you declare each part to be a copy of a subset of another object. This would allow delayed commit rename() operations; committers could

          good

          • can take any blob and rename it anywhere in the FS
          • with parallelised PUT calls, copy time is ~ filesize/(parts * 6e6) seconds, same as parallel renames today.
          • could be retrofitted to existing committers which rely on renames

          bad

          • still need a story for propagating commit metadata from tasks to job committer.
          • still need a story for cleaning up failed tasks and jobs
          • still not O(1) in the commit

          This could be used in the bit of the spark committer which can move files to absolute locations; tasks would do the copy and send the serialized commit data to the job committer. if the path _temporary/$attemptId/$taskId is used for the work output, then the cleanup mechanism of all work prior to the copy on task/job failure is the same as today's output committer. That doesn't cover the pending commit though, unless again something is saved to the FS somewhere. Again, something could be explicitly PUT into _pending/$jobId/$taskId for this, just save the pending data to a file with a UUID and both commit and abort can find and use it.

          Proposed: add this once the core commit mech is done

          Show
          stevel@apache.org Steve Loughran added a comment - I've realised we can also do delayed-commit object COPY too, because multipart PUT lets you declare each part to be a copy of a subset of another object. This would allow delayed commit rename() operations; committers could good can take any blob and rename it anywhere in the FS with parallelised PUT calls, copy time is ~ filesize/(parts * 6e6) seconds, same as parallel renames today. could be retrofitted to existing committers which rely on renames bad still need a story for propagating commit metadata from tasks to job committer. still need a story for cleaning up failed tasks and jobs still not O(1) in the commit This could be used in the bit of the spark committer which can move files to absolute locations; tasks would do the copy and send the serialized commit data to the job committer. if the path _temporary/$attemptId/$taskId is used for the work output, then the cleanup mechanism of all work prior to the copy on task/job failure is the same as today's output committer. That doesn't cover the pending commit though, unless again something is saved to the FS somewhere. Again, something could be explicitly PUT into _pending/$jobId/$taskId for this, just save the pending data to a file with a UUID and both commit and abort can find and use it. Proposed: add this once the core commit mech is done
          Hide
          stevel@apache.org Steve Loughran added a comment -
          Show
          stevel@apache.org Steve Loughran added a comment - Netflix committer: https://github.com/rdblue/s3committer
          Hide
          stevel@apache.org Steve Loughran added a comment -

          Patch 007; (limited) tests now working.

          I'm changing direction slightly here and working on making the first committer a derivative of the Netflix Committer. This stages to the local filesystem, then, in task commit, uploads the generated files as the multipart PUT; co-ordination information is persisted via HDFS. While this appears to add some complexity to the writing process, it avoids "magic" in the filesystem, and, by using HDFS, doesn't need dynamo DB.

          What it also adds is: actual use in production, along with minicluster tests. Production use is going to mean that resilience to failures and odd execution orderings are more likely to have been addressed; with my own committer I'd be relearning how things fail.

          Accordingly, I think it'd be more likely to be ready for use.

          Patch 007 doesn't include any of that, it's the "before" patch.

          I'm now merging in the netflix code, using S3A and the WriteOperationHelper as the means of talking to S3. Their code is ASF licensed, but the copyright headers still say Netflix...we need it to be added to this JIRA as a patch before we could think about committing to the ASF codebase. In the meantime, I'll work on it locally

          Show
          stevel@apache.org Steve Loughran added a comment - Patch 007; (limited) tests now working. I'm changing direction slightly here and working on making the first committer a derivative of the Netflix Committer . This stages to the local filesystem, then, in task commit, uploads the generated files as the multipart PUT; co-ordination information is persisted via HDFS. While this appears to add some complexity to the writing process, it avoids "magic" in the filesystem, and, by using HDFS, doesn't need dynamo DB. What it also adds is: actual use in production, along with minicluster tests. Production use is going to mean that resilience to failures and odd execution orderings are more likely to have been addressed; with my own committer I'd be relearning how things fail. Accordingly, I think it'd be more likely to be ready for use. Patch 007 doesn't include any of that, it's the "before" patch. I'm now merging in the netflix code, using S3A and the WriteOperationHelper as the means of talking to S3. Their code is ASF licensed, but the copyright headers still say Netflix...we need it to be added to this JIRA as a patch before we could think about committing to the ASF codebase. In the meantime, I'll work on it locally
          Hide
          rdblue Ryan Blue added a comment -

          I'm attaching the S3 committer code to show intent to contribute.

          Show
          rdblue Ryan Blue added a comment - I'm attaching the S3 committer code to show intent to contribute.
          Hide
          stevel@apache.org Steve Loughran added a comment -

          Thanks! I'll share what I've done with it soon, which currently comes down to: pulling into hadoop-aws module, moving under org.apache.hadoop.fs.s3a.commit, reparenting it to the new PathOutputCommitter (i.e: no longer a subclass of FileOutputCommitter, for better or worse), breaking all the tests. Related to that reparenting, see.

          Show
          stevel@apache.org Steve Loughran added a comment - Thanks! I'll share what I've done with it soon, which currently comes down to: pulling into hadoop-aws module, moving under org.apache.hadoop.fs.s3a.commit, reparenting it to the new PathOutputCommitter (i.e: no longer a subclass of FileOutputCommitter, for better or worse), breaking all the tests. Related to that reparenting, see.
          Hide
          rdblue Ryan Blue added a comment -

          Thanks for doing so much to get this in! Let me know when it is a good time to have a look at it or if you need anything. There are a few extension points that we rely on that I'd like to make sure are kept, like getFinalOutputPath that I would like to keep if possible.

          Show
          rdblue Ryan Blue added a comment - Thanks for doing so much to get this in! Let me know when it is a good time to have a look at it or if you need anything. There are a few extension points that we rely on that I'd like to make sure are kept, like getFinalOutputPath that I would like to keep if possible.
          Hide
          stevel@apache.org Steve Loughran added a comment -

          Ryan, I think you'll find this is inextricably linked with S3a now; I'll leave that hook in, but it'll ultimately be tied down. Why? I want to lock all access down to operations in S3AFS, rather than let you get at the Client. Do that and not only do the metrics & stuff get lost, but s3guard won't like it.

          Show
          stevel@apache.org Steve Loughran added a comment - Ryan, I think you'll find this is inextricably linked with S3a now; I'll leave that hook in, but it'll ultimately be tied down. Why? I want to lock all access down to operations in S3AFS, rather than let you get at the Client. Do that and not only do the metrics & stuff get lost, but s3guard won't like it.
          Hide
          stevel@apache.org Steve Loughran added a comment -

          Patch 009; pulling in Ryan's netflix committer; I've called in "the Staging committer", with the one playing games in S3a "the magic committer"; that one I've actually put to one side as an "eventually" feature; the goal being: common code & data underneath, but different strategies of getting the data up.

          Details at the end of: https://github.com/steveloughran/hadoop/blob/s3guard/HADOOP-13786-committer/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/s3a_committer.md

          It uses the existing FileOutputCommitter for its work; some time was spent in a debugger until I worked out that the code relies on Algorithm 1, not the current default of "2". Ideally I'd rip out the FOC altogether, it's just a complication once each task only generates a single file whose name can be guaranteed to be unique. Task commits copy into the job dir; job commits rename to dest, files are scanned and used to generate the multipart commits for the real dest. Leaving alone for now.

          • All of Ryan's tests are failing as the FS mocking is being rejected "not an s3a filesystem"'; more migration work needed there.
          • I've been working on the protocol integration test copied out from the mapreduce module; this tests various sequences of the operations and asserts about final outcomes. Some of them are starting to work (core commit and abort), but not those with subdirectories, such as the intermediate Map output.
          Show
          stevel@apache.org Steve Loughran added a comment - Patch 009; pulling in Ryan's netflix committer; I've called in "the Staging committer", with the one playing games in S3a "the magic committer"; that one I've actually put to one side as an "eventually" feature; the goal being: common code & data underneath, but different strategies of getting the data up. Details at the end of: https://github.com/steveloughran/hadoop/blob/s3guard/HADOOP-13786-committer/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/s3a_committer.md It uses the existing FileOutputCommitter for its work; some time was spent in a debugger until I worked out that the code relies on Algorithm 1, not the current default of "2". Ideally I'd rip out the FOC altogether, it's just a complication once each task only generates a single file whose name can be guaranteed to be unique. Task commits copy into the job dir; job commits rename to dest, files are scanned and used to generate the multipart commits for the real dest. Leaving alone for now. All of Ryan's tests are failing as the FS mocking is being rejected "not an s3a filesystem"'; more migration work needed there. I've been working on the protocol integration test copied out from the mapreduce module; this tests various sequences of the operations and asserts about final outcomes. Some of them are starting to work (core commit and abort), but not those with subdirectories, such as the intermediate Map output.
          Hide
          stevel@apache.org Steve Loughran added a comment -

          I was expecting a veto from yetus

          Show
          stevel@apache.org Steve Loughran added a comment - I was expecting a veto from yetus
          Hide
          rdblue Ryan Blue added a comment -

          Is there a branch where I can take a look at the S3A test issue? I can probably get them working.

          Show
          rdblue Ryan Blue added a comment - Is there a branch where I can take a look at the S3A test issue? I can probably get them working.
          Hide
          stevel@apache.org Steve Loughran added a comment -

          Right now I'm doing this out of an ASF branch, rebasing onto the HADOOP-13345 branch regularly. That way I can do things without adult supervision

          https://github.com/steveloughran/hadoop/tree/s3guard/HADOOP-13786-committer

          Test wise, I've been focusing on the integration tests org.apache.hadoop.fs.s3a.commit.staging.ITestStagingCommitProtocol, which is derived from org.apache.hadoop.mapreduce.lib.output.TestFileOutputCommitter ; because they're from there they form part of the expectations of the protocol implementation for commitment
          (more precisely, they're the closest we have to any definition). All these intiial tests work, except for those generating files in a subdirectory, e.g. part-0000/subfile; something critical for the intermediate output of an MR job.

          The scanner for files to upload is just doing a flat list and then getting into trouble when it gets handed a directory to upload instead of a simple file.

          java.io.FileNotFoundException: Not a file/Users/stevel/Projects/hadoop-trunk/hadoop-tools/hadoop-aws/target/tmp/mapred/local/job_200707121733_0001/_temporary/0/_temporary/attempt_200707121733_0001_m_000000_0/part-m-00000
          
          	at org.apache.hadoop.fs.s3a.commit.staging.S3Util.multipartUpload(S3Util.java:104)
          	at org.apache.hadoop.fs.s3a.commit.staging.StagingS3GuardCommitter$8.run(StagingS3GuardCommitter.java:784)
          	at org.apache.hadoop.fs.s3a.commit.staging.StagingS3GuardCommitter$8.run(StagingS3GuardCommitter.java:771)
          	at org.apache.hadoop.fs.s3a.commit.staging.Tasks$Builder.runSingleThreaded(Tasks.java:122)
          	at org.apache.hadoop.fs.s3a.commit.staging.Tasks$Builder.run(Tasks.java:108)
          	at org.apache.hadoop.fs.s3a.commit.staging.StagingS3GuardCommitter.commitTaskInternal(StagingS3GuardCommitter.java:771)
          	at org.apache.hadoop.fs.s3a.commit.staging.StagingS3GuardCommitter.commitTask(StagingS3GuardCommitter.java:716)
          	at org.apache.hadoop.fs.s3a.commit.AbstractITCommitProtocol.commit(AbstractITCommitProtocol.java:684)
          	at org.apache.hadoop.fs.s3a.commit.AbstractITCommitProtocol.testMapFileOutputCommitter(AbstractITCommitProtocol.java:567)
          	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.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:47)
          	at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12)
          	at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:44)
          	at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17)
          	at org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26)
          	at org.junit.internal.runners.statements.RunAfters.evaluate(RunAfters.java:27)
          	at org.junit.rules.TestWatcher$1.evaluate(TestWatcher.java:55)
          	at org.junit.internal.runners.statements.FailOnTimeout$StatementThread.run(FailOnTimeout.java:74)
          

          What I need to do is go from the flat listFiles() to the recursive one, then use that to create the offset of the final destination.

          Regarding the mock tests, it's all happening because bits of the code now expect and S3aFS, and its only a base FileSystem, with a limited set of operations, being mocked

          It's not going to be quite enough to mock S3AFS, unless those extra methods come in (which will surface as we try).

          FWIW, I'd actually prefer that, wherever possible, real integration tests were used over mock ones. Yes, they are slower, no, yetus and jenkins don't run them, but because they really do test the endpoint, and will catch regressions in the S3A client itself, quirks in different S3 implementations, etc. Given you've written them, it'll be good to get working. And the fault generation is great to test the resilience of the committer.

          Show
          stevel@apache.org Steve Loughran added a comment - Right now I'm doing this out of an ASF branch, rebasing onto the HADOOP-13345 branch regularly. That way I can do things without adult supervision https://github.com/steveloughran/hadoop/tree/s3guard/HADOOP-13786-committer Test wise, I've been focusing on the integration tests org.apache.hadoop.fs.s3a.commit.staging.ITestStagingCommitProtocol , which is derived from org.apache.hadoop.mapreduce.lib.output.TestFileOutputCommitter ; because they're from there they form part of the expectations of the protocol implementation for commitment (more precisely, they're the closest we have to any definition). All these intiial tests work, except for those generating files in a subdirectory, e.g. part-0000/subfile ; something critical for the intermediate output of an MR job. The scanner for files to upload is just doing a flat list and then getting into trouble when it gets handed a directory to upload instead of a simple file. java.io.FileNotFoundException: Not a file/Users/stevel/Projects/hadoop-trunk/hadoop-tools/hadoop-aws/target/tmp/mapred/local/job_200707121733_0001/_temporary/0/_temporary/attempt_200707121733_0001_m_000000_0/part-m-00000 at org.apache.hadoop.fs.s3a.commit.staging.S3Util.multipartUpload(S3Util.java:104) at org.apache.hadoop.fs.s3a.commit.staging.StagingS3GuardCommitter$8.run(StagingS3GuardCommitter.java:784) at org.apache.hadoop.fs.s3a.commit.staging.StagingS3GuardCommitter$8.run(StagingS3GuardCommitter.java:771) at org.apache.hadoop.fs.s3a.commit.staging.Tasks$Builder.runSingleThreaded(Tasks.java:122) at org.apache.hadoop.fs.s3a.commit.staging.Tasks$Builder.run(Tasks.java:108) at org.apache.hadoop.fs.s3a.commit.staging.StagingS3GuardCommitter.commitTaskInternal(StagingS3GuardCommitter.java:771) at org.apache.hadoop.fs.s3a.commit.staging.StagingS3GuardCommitter.commitTask(StagingS3GuardCommitter.java:716) at org.apache.hadoop.fs.s3a.commit.AbstractITCommitProtocol.commit(AbstractITCommitProtocol.java:684) at org.apache.hadoop.fs.s3a.commit.AbstractITCommitProtocol.testMapFileOutputCommitter(AbstractITCommitProtocol.java:567) 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.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:47) at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12) at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:44) at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17) at org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26) at org.junit.internal.runners.statements.RunAfters.evaluate(RunAfters.java:27) at org.junit.rules.TestWatcher$1.evaluate(TestWatcher.java:55) at org.junit.internal.runners.statements.FailOnTimeout$StatementThread.run(FailOnTimeout.java:74) What I need to do is go from the flat listFiles() to the recursive one, then use that to create the offset of the final destination. Regarding the mock tests, it's all happening because bits of the code now expect and S3aFS, and its only a base FileSystem, with a limited set of operations, being mocked It's not going to be quite enough to mock S3AFS, unless those extra methods come in (which will surface as we try). FWIW, I'd actually prefer that, wherever possible, real integration tests were used over mock ones. Yes, they are slower, no, yetus and jenkins don't run them, but because they really do test the endpoint, and will catch regressions in the S3A client itself, quirks in different S3 implementations, etc. Given you've written them, it'll be good to get working. And the fault generation is great to test the resilience of the committer.
          Hide
          stevel@apache.org Steve Loughran added a comment -

          Patch 010; work on the mock tests. Overall, ~50% success rate.

          These tests are failing for two main reasons

          1. I've broken the code
          2. In changing the code, I've broken the test

          A fair few of the tests are failing in that the mock calls don't follow the path they've expected; that's the problem with mocks...you are asserting about the internal operations, rather than the final observed state of the SUT. Sometimes that's good for looking into the interior, but it's very, very brittle.

          These tests are all doing something ugly to get a mock s3a FS set up for code to get when they ask for an FS. I plan to remove that wrapper mock and inject whatever mock FS is created straight into the FileSystem.get() cache. That's the proper way to do it.

          After that, I'll look at why the tests are failing, focusing on ones where results are not what is expected, rather than just mock counter mismatch. I'll assume those are false alarms for now, and only worry about the details once the more functional tests have passed

          Show
          stevel@apache.org Steve Loughran added a comment - Patch 010; work on the mock tests. Overall, ~50% success rate. These tests are failing for two main reasons I've broken the code In changing the code, I've broken the test A fair few of the tests are failing in that the mock calls don't follow the path they've expected; that's the problem with mocks...you are asserting about the internal operations, rather than the final observed state of the SUT. Sometimes that's good for looking into the interior, but it's very, very brittle. These tests are all doing something ugly to get a mock s3a FS set up for code to get when they ask for an FS. I plan to remove that wrapper mock and inject whatever mock FS is created straight into the FileSystem.get() cache. That's the proper way to do it. After that, I'll look at why the tests are failing, focusing on ones where results are not what is expected, rather than just mock counter mismatch. I'll assume those are false alarms for now, and only worry about the details once the more functional tests have passed
          Hide
          Thomas Demoor Thomas Demoor added a comment -

          Hi Ryan, thanks for contributing to this effort.

          Both approaches have the same concept of having data persistence but postponing object visibility through holding back on the multipart commit. +1 on sharing common code and having 2 committers.

          Thoughts after my first pass, on 009.patch:

          StagingCommitter:

          • I can see how this is ideal in a pre s3ablockoutputstream world. Using the stage to disk allows code which is conceptually similar to the exisiting filoutputcommitter
          • +: well tested and lots of failure handling (crash during overwrite protection.)
          • +: flexible, we could still use s3a as the wrappedCommiter for s3guard / consistent s3 implementations and thus not require a hdfs cluster to be around
          • -: s3client in committer itself (another threadpool, will they fight? )
          • -: staging to local disk not essentail and can cause issues (disk out of space, disk throughput limitation = network is faster, large files ...) = one of the reasons we made the blockoutputstreams

          MagicCommiter:

          • +: less moving parts (no local disk, no second committer for the metadata)
          • +: s3 complexity solved by reusing s3ablockoutputstream
          • -: less mileage / testing: if things break where will there be garbage?
          • Saw quite some unused variables, methods, etc. but I considered this work in progress so /ignore
          • There are some extra HEAD requests to verify that things really happened which I think are superfluous. For instance, org/apache/hadoop/fs/s3a/commit/FileCommitActions.java:78 the HEAD after PUT. Is this to get around eventual consistency? This is head after put, so should be consistent even on aws. Maybe useful for PUT overwrite but then you also need to check the ETag. I think this HEAD should be optional behind some config flag.

          Varia:

          • Don't think the default (hdfs) workflow has crash-during-jobcommit protection (fileoutputcommitter algo v2 already makes files visible in taskcommit afaik) , so not sure it is required for our committers.
          Show
          Thomas Demoor Thomas Demoor added a comment - Hi Ryan, thanks for contributing to this effort. Both approaches have the same concept of having data persistence but postponing object visibility through holding back on the multipart commit. +1 on sharing common code and having 2 committers. Thoughts after my first pass, on 009.patch: StagingCommitter: I can see how this is ideal in a pre s3ablockoutputstream world. Using the stage to disk allows code which is conceptually similar to the exisiting filoutputcommitter +: well tested and lots of failure handling (crash during overwrite protection.) +: flexible, we could still use s3a as the wrappedCommiter for s3guard / consistent s3 implementations and thus not require a hdfs cluster to be around -: s3client in committer itself (another threadpool, will they fight? ) -: staging to local disk not essentail and can cause issues (disk out of space, disk throughput limitation = network is faster, large files ...) = one of the reasons we made the blockoutputstreams MagicCommiter: +: less moving parts (no local disk, no second committer for the metadata) +: s3 complexity solved by reusing s3ablockoutputstream -: less mileage / testing: if things break where will there be garbage? Saw quite some unused variables, methods, etc. but I considered this work in progress so /ignore There are some extra HEAD requests to verify that things really happened which I think are superfluous. For instance, org/apache/hadoop/fs/s3a/commit/FileCommitActions.java:78 the HEAD after PUT. Is this to get around eventual consistency? This is head after put, so should be consistent even on aws. Maybe useful for PUT overwrite but then you also need to check the ETag. I think this HEAD should be optional behind some config flag. Varia: Don't think the default (hdfs) workflow has crash-during-jobcommit protection (fileoutputcommitter algo v2 already makes files visible in taskcommit afaik) , so not sure it is required for our committers.
          Hide
          rdblue Ryan Blue added a comment -

          For the staging committer drawbacks, I think there's a clear path to avoid them.

          The committer is not intended to instantiate its own S3Client. It does for testing, but when it is integrated with S3A it should be passed a configured client when it is instantiated, or should use package-local access to get one from the S3A FS object. In other words, the default findClient method shouldn't be used; we don't use it other than for testing. My intent was for S3A to have a FileSystem#newOutputCommitter(Path, JobContext) factory method. That way, the FS can pass its internal S3 client instead of instantiating two.

          The storage on local disk isn't a requirement. We can replace that with an output stream that buffers in memory and sends parts to S3 when they are ready (we're planning on doing this eventually). This is just waiting on a stable API to rely on that can close a stream, but not commit data. Since the committer API right now expects tasks to create files underneath the work path, we'll have to figure out how tasks can get a multi-part stream that is committed later without using a different method.

          We can also pass in a thread-pool if there is a better one to use. I think this is separate enough that it should be easy.

          Show
          rdblue Ryan Blue added a comment - For the staging committer drawbacks, I think there's a clear path to avoid them. The committer is not intended to instantiate its own S3Client. It does for testing, but when it is integrated with S3A it should be passed a configured client when it is instantiated, or should use package-local access to get one from the S3A FS object. In other words, the default findClient method shouldn't be used; we don't use it other than for testing. My intent was for S3A to have a FileSystem#newOutputCommitter(Path, JobContext) factory method. That way, the FS can pass its internal S3 client instead of instantiating two. The storage on local disk isn't a requirement. We can replace that with an output stream that buffers in memory and sends parts to S3 when they are ready (we're planning on doing this eventually). This is just waiting on a stable API to rely on that can close a stream, but not commit data. Since the committer API right now expects tasks to create files underneath the work path, we'll have to figure out how tasks can get a multi-part stream that is committed later without using a different method. We can also pass in a thread-pool if there is a better one to use. I think this is separate enough that it should be easy.
          Hide
          stevel@apache.org Steve Loughran added a comment -

          Patch 011; lot more of the tests are working. More specifically, two causes of failure.

          the tests which are using mockito to its fullest, TestStagingDirectoryOutputCommitter and TestStagingPartitionedJobCommit are failing because the FS operations have changed; some differences in which methods get called and when. They're going to have be synced up and then maintained. It's a pain, but it does let the tests run even without credentials.

          The tests which expect the names of the output to be part-0000-$UUID. I turned that off yesterday while getting the "commit protocol" integration test to work; it doesn't do that, though I know sometimes spark jobs do for that unique output.

          I don't know what to do here? Hard code "no uuid suffix" & change the tests, probably break ryan's code, not suit other people. Hard code "uuid suffix:, some things work, others get confused. Or go for the option, double your test space.

          This is somewhere where we'll need to call in the people who understand the internals of commitment and its integration

          Show
          stevel@apache.org Steve Loughran added a comment - Patch 011; lot more of the tests are working. More specifically, two causes of failure. the tests which are using mockito to its fullest, TestStagingDirectoryOutputCommitter and TestStagingPartitionedJobCommit are failing because the FS operations have changed; some differences in which methods get called and when. They're going to have be synced up and then maintained. It's a pain, but it does let the tests run even without credentials. The tests which expect the names of the output to be part-0000-$UUID. I turned that off yesterday while getting the "commit protocol" integration test to work; it doesn't do that, though I know sometimes spark jobs do for that unique output. I don't know what to do here? Hard code "no uuid suffix" & change the tests, probably break ryan's code, not suit other people. Hard code "uuid suffix:, some things work, others get confused. Or go for the option, double your test space. This is somewhere where we'll need to call in the people who understand the internals of commitment and its integration
          Hide
          stevel@apache.org Steve Loughran added a comment -
          1. FileCommitActions is my code. Thomas: blame me there. Sanity check by the look of things. Ryan's committer has a near-identical persistent data structure of pending commit information, different commit codepath. It can move over to FileCommitActions once PersistentCommitData also supports ser/deser of a list of commits to a (JSON) file, rather than just one file/pending commit
          2. we are going to have to move sight of the S3Client away from the committers entirely, so that s3guard can stay in sync with what's happening. Otherwise a caller can use the client to complete the put, but s3guard won't know to update its tables. S3aFileSystem.WriteOperationHelper has everything needed, or, if it doesn't, can add the rest. I've not gone near that yet as getting tests working comes ahead of getting the integration complete.
          3. Again, threadpools: no real opinion. The bulk uploads should be assigned to the "slow operations pool" on the FS and if we have a "faster ops pool", all the abort/commit calls in there. I do like that whole Threads design BTW, very nice, sliding in to a java 8+ world nicely.
          Show
          stevel@apache.org Steve Loughran added a comment - FileCommitActions is my code. Thomas: blame me there. Sanity check by the look of things. Ryan's committer has a near-identical persistent data structure of pending commit information, different commit codepath. It can move over to FileCommitActions once PersistentCommitData also supports ser/deser of a list of commits to a (JSON) file, rather than just one file/pending commit we are going to have to move sight of the S3Client away from the committers entirely, so that s3guard can stay in sync with what's happening. Otherwise a caller can use the client to complete the put, but s3guard won't know to update its tables. S3aFileSystem.WriteOperationHelper has everything needed, or, if it doesn't, can add the rest. I've not gone near that yet as getting tests working comes ahead of getting the integration complete. Again, threadpools: no real opinion. The bulk uploads should be assigned to the "slow operations pool" on the FS and if we have a "faster ops pool", all the abort/commit calls in there. I do like that whole Threads design BTW, very nice, sliding in to a java 8+ world nicely.
          Hide
          rdblue Ryan Blue added a comment -

          On UUID suffixes: the option is needed for the "append" conflict resolution. Without it, you can easily overwrite files from previous writes. We also use it to identify files across partitions added in the same batch. I think it is worth keeping if this committer is to be used for more than just creating or replacing a single directory of files.

          Show
          rdblue Ryan Blue added a comment - On UUID suffixes: the option is needed for the "append" conflict resolution. Without it, you can easily overwrite files from previous writes. We also use it to identify files across partitions added in the same batch. I think it is worth keeping if this committer is to be used for more than just creating or replacing a single directory of files.
          Hide
          stevel@apache.org Steve Loughran added a comment -

          OK, I'll keep as an on/off feature, on by default, off for some of the tests. The UUID is also needed for temp file placement.

          On that topic, I've tweaked temp file setup for both testing (use java.io.tmpdir which will be under target) and on HDFS. Really, every FS should have a .getTempDir() method to return a temp dir guaranteed to have R/W access by a specific user.

          Show
          stevel@apache.org Steve Loughran added a comment - OK, I'll keep as an on/off feature, on by default, off for some of the tests. The UUID is also needed for temp file placement. On that topic, I've tweaked temp file setup for both testing (use java.io.tmpdir which will be under target) and on HDFS. Really, every FS should have a .getTempDir() method to return a temp dir guaranteed to have R/W access by a specific user.
          Hide
          stevel@apache.org Steve Loughran added a comment - - edited

          patch 012:

          • UUID reinstated by default; you can turn this off.
          • mostly all the unit tests are passing,
          • the mocks in TestStagingPartitionedJobCommit, TestStagingDirectoryOutputCommitter are having an unexpected delete in mock invocations (good? bad?), and because the staging committer doesn't yet handle directories, some of the protocol tests are failing.
          • Added AbstractITCommitProtocol subclasses for the directory and partition committers
          • Those in the protocol IT tests which are failing on the job/commit fail are failing as the tests aren't looking for the right exception.

          One thing to highlight here is that when running these tests from my desktop, the staging commits seem to be faster than the magic ones. Why? A lot less S3 communication over a long haul link during task setup/commit, and there's no real data to upload, so the cost delaying the upload until the task commit is negligible.

          Show
          stevel@apache.org Steve Loughran added a comment - - edited patch 012: UUID reinstated by default; you can turn this off. mostly all the unit tests are passing, the mocks in TestStagingPartitionedJobCommit , TestStagingDirectoryOutputCommitter are having an unexpected delete in mock invocations (good? bad?), and because the staging committer doesn't yet handle directories, some of the protocol tests are failing. Added AbstractITCommitProtocol subclasses for the directory and partition committers Those in the protocol IT tests which are failing on the job/commit fail are failing as the tests aren't looking for the right exception. One thing to highlight here is that when running these tests from my desktop, the staging commits seem to be faster than the magic ones. Why? A lot less S3 communication over a long haul link during task setup/commit, and there's no real data to upload, so the cost delaying the upload until the task commit is negligible.
          Hide
          stevel@apache.org Steve Loughran added a comment -

          HADOOP-13786 patch 013.

          All mock tests are working.

          More specifically, they were got working, then the commit logic tuned to reduce the number of S3 calls

          • no exists(path) check before a delete() if the policy is replace
          • no checking for return code of delete(), as we know it never signals an error, merely that the destination path didn't exist at the time of call.

          The tests have also been tuned to be a bit more explicit about what they are declaring and asserting; less repetition in mock object setup.

          Also: ability to turn up logging of mock operations, including stack trace level of invocation. Useful to work out things like why more delete() calls are made than expected.

          Most of the committer IT tests are working

          Everything is working except the IT protocol tests testMapFileOutputCommitter and testConcurrentCommitTaskWithSubDir, which expect directories to be handled. I will do that at least for the Directory Committer, with some mock tests as well as a fixed IT test, and skip them in the partition committer

          LambdaTestUtils tuning

          Ryan's patch had some assertThrown() assertions which I've been moving to the common test base.

          While intercept() has some features assertThrown() lacked, it doesn't support handing down extra diagnostics messages. Fixed. We could go one step further and allow callers to provide a closure () -> String for diagnostics, perhaps, though maybe we can wait to see what JUnit 5 has first

          TODO

          • directory trees in the directory committer
          • move to direct API calls on S3A
          • when s3guard is enabled, make sure PUT commits are updating the entire metastore tree.
          Show
          stevel@apache.org Steve Loughran added a comment - HADOOP-13786 patch 013. All mock tests are working. More specifically, they were got working, then the commit logic tuned to reduce the number of S3 calls no exists(path) check before a delete() if the policy is replace no checking for return code of delete() , as we know it never signals an error, merely that the destination path didn't exist at the time of call. The tests have also been tuned to be a bit more explicit about what they are declaring and asserting; less repetition in mock object setup. Also: ability to turn up logging of mock operations, including stack trace level of invocation. Useful to work out things like why more delete() calls are made than expected. Most of the committer IT tests are working Everything is working except the IT protocol tests testMapFileOutputCommitter and testConcurrentCommitTaskWithSubDir , which expect directories to be handled. I will do that at least for the Directory Committer, with some mock tests as well as a fixed IT test, and skip them in the partition committer LambdaTestUtils tuning Ryan's patch had some assertThrown() assertions which I've been moving to the common test base. While intercept() has some features assertThrown() lacked, it doesn't support handing down extra diagnostics messages. Fixed. We could go one step further and allow callers to provide a closure () -> String for diagnostics, perhaps, though maybe we can wait to see what JUnit 5 has first TODO directory trees in the directory committer move to direct API calls on S3A when s3guard is enabled, make sure PUT commits are updating the entire metastore tree.
          Hide
          hadoopqa Hadoop QA added a comment -
          -1 overall



          Vote Subsystem Runtime Comment
          0 reexec 0m 18s Docker mode activated.
          +1 @author 0m 0s The patch does not contain any @author tags.
          +1 test4tests 0m 0s The patch appears to include 29 new or modified test files.
          0 mvndep 2m 25s Maven dependency ordering for branch
          +1 mvninstall 16m 42s HADOOP-13345 passed
          -1 compile 7m 53s root in HADOOP-13345 failed.
          +1 checkstyle 2m 10s HADOOP-13345 passed
          +1 mvnsite 2m 29s HADOOP-13345 passed
          +1 mvneclipse 2m 38s HADOOP-13345 passed
          +1 findbugs 3m 34s HADOOP-13345 passed
          +1 javadoc 2m 5s HADOOP-13345 passed
          0 mvndep 0m 17s Maven dependency ordering for patch
          +1 mvninstall 1m 34s the patch passed
          -1 compile 7m 12s root in the patch failed.
          -1 javac 7m 12s root in the patch failed.
          -0 checkstyle 2m 14s root: The patch generated 255 new + 99 unchanged - 14 fixed = 354 total (was 113)
          -1 mvnsite 0m 40s hadoop-aws in the patch failed.
          +1 mvneclipse 1m 31s the patch passed
          -1 whitespace 0m 0s The patch has 17 line(s) that end in whitespace. Use git apply --whitespace=fix <<patch_file>>. Refer https://git-scm.com/docs/git-apply
          +1 xml 0m 1s The patch has no ill-formed XML file.
          +1 findbugs 4m 2s the patch passed
          -1 javadoc 0m 38s hadoop-mapreduce-project_hadoop-mapreduce-client_hadoop-mapreduce-client-core generated 1 new + 2496 unchanged - 0 fixed = 2497 total (was 2496)
          -1 javadoc 0m 32s hadoop-aws in the patch failed.
          +1 unit 9m 29s hadoop-common in the patch passed.
          +1 unit 3m 11s hadoop-mapreduce-client-core in the patch passed.
          -1 unit 1m 45s hadoop-aws in the patch failed.
          +1 asflicense 0m 52s The patch does not generate ASF License warnings.
          102m 31s



          Reason Tests
          Failed junit tests hadoop.fs.s3a.commit.staging.TestStagingPartitionedFileListing
            hadoop.fs.s3a.commit.staging.TestStagingMRJob



          Subsystem Report/Notes
          Docker Image:yetus/hadoop:a9ad5d6
          JIRA Issue HADOOP-13786
          JIRA Patch URL https://issues.apache.org/jira/secure/attachment/12858766/HADOOP-13786-HADOOP-13345-013.patch
          Optional Tests asflicense compile javac javadoc mvninstall mvnsite unit findbugs checkstyle xml
          uname Linux facb1214e3f1 3.13.0-106-generic #153-Ubuntu SMP Tue Dec 6 15:44:32 UTC 2016 x86_64 x86_64 x86_64 GNU/Linux
          Build tool maven
          Personality /testptch/hadoop/patchprocess/precommit/personality/provided.sh
          git revision HADOOP-13345 / 566c8b7
          Default Java 1.8.0_121
          compile https://builds.apache.org/job/PreCommit-HADOOP-Build/11824/artifact/patchprocess/branch-compile-root.txt
          findbugs v3.0.0
          compile https://builds.apache.org/job/PreCommit-HADOOP-Build/11824/artifact/patchprocess/patch-compile-root.txt
          javac https://builds.apache.org/job/PreCommit-HADOOP-Build/11824/artifact/patchprocess/patch-compile-root.txt
          checkstyle https://builds.apache.org/job/PreCommit-HADOOP-Build/11824/artifact/patchprocess/diff-checkstyle-root.txt
          mvnsite https://builds.apache.org/job/PreCommit-HADOOP-Build/11824/artifact/patchprocess/patch-mvnsite-hadoop-tools_hadoop-aws.txt
          whitespace https://builds.apache.org/job/PreCommit-HADOOP-Build/11824/artifact/patchprocess/whitespace-eol.txt
          javadoc https://builds.apache.org/job/PreCommit-HADOOP-Build/11824/artifact/patchprocess/diff-javadoc-javadoc-hadoop-mapreduce-project_hadoop-mapreduce-client_hadoop-mapreduce-client-core.txt
          javadoc https://builds.apache.org/job/PreCommit-HADOOP-Build/11824/artifact/patchprocess/patch-javadoc-hadoop-tools_hadoop-aws.txt
          unit https://builds.apache.org/job/PreCommit-HADOOP-Build/11824/artifact/patchprocess/patch-unit-hadoop-tools_hadoop-aws.txt
          Test Results https://builds.apache.org/job/PreCommit-HADOOP-Build/11824/testReport/
          modules C: hadoop-common-project/hadoop-common hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core hadoop-tools/hadoop-aws U: .
          Console output https://builds.apache.org/job/PreCommit-HADOOP-Build/11824/console
          Powered by Apache Yetus 0.5.0-SNAPSHOT http://yetus.apache.org

          This message was automatically generated.

          Show
          hadoopqa Hadoop QA added a comment - -1 overall Vote Subsystem Runtime Comment 0 reexec 0m 18s Docker mode activated. +1 @author 0m 0s The patch does not contain any @author tags. +1 test4tests 0m 0s The patch appears to include 29 new or modified test files. 0 mvndep 2m 25s Maven dependency ordering for branch +1 mvninstall 16m 42s HADOOP-13345 passed -1 compile 7m 53s root in HADOOP-13345 failed. +1 checkstyle 2m 10s HADOOP-13345 passed +1 mvnsite 2m 29s HADOOP-13345 passed +1 mvneclipse 2m 38s HADOOP-13345 passed +1 findbugs 3m 34s HADOOP-13345 passed +1 javadoc 2m 5s HADOOP-13345 passed 0 mvndep 0m 17s Maven dependency ordering for patch +1 mvninstall 1m 34s the patch passed -1 compile 7m 12s root in the patch failed. -1 javac 7m 12s root in the patch failed. -0 checkstyle 2m 14s root: The patch generated 255 new + 99 unchanged - 14 fixed = 354 total (was 113) -1 mvnsite 0m 40s hadoop-aws in the patch failed. +1 mvneclipse 1m 31s the patch passed -1 whitespace 0m 0s The patch has 17 line(s) that end in whitespace. Use git apply --whitespace=fix <<patch_file>>. Refer https://git-scm.com/docs/git-apply +1 xml 0m 1s The patch has no ill-formed XML file. +1 findbugs 4m 2s the patch passed -1 javadoc 0m 38s hadoop-mapreduce-project_hadoop-mapreduce-client_hadoop-mapreduce-client-core generated 1 new + 2496 unchanged - 0 fixed = 2497 total (was 2496) -1 javadoc 0m 32s hadoop-aws in the patch failed. +1 unit 9m 29s hadoop-common in the patch passed. +1 unit 3m 11s hadoop-mapreduce-client-core in the patch passed. -1 unit 1m 45s hadoop-aws in the patch failed. +1 asflicense 0m 52s The patch does not generate ASF License warnings. 102m 31s Reason Tests Failed junit tests hadoop.fs.s3a.commit.staging.TestStagingPartitionedFileListing   hadoop.fs.s3a.commit.staging.TestStagingMRJob Subsystem Report/Notes Docker Image:yetus/hadoop:a9ad5d6 JIRA Issue HADOOP-13786 JIRA Patch URL https://issues.apache.org/jira/secure/attachment/12858766/HADOOP-13786-HADOOP-13345-013.patch Optional Tests asflicense compile javac javadoc mvninstall mvnsite unit findbugs checkstyle xml uname Linux facb1214e3f1 3.13.0-106-generic #153-Ubuntu SMP Tue Dec 6 15:44:32 UTC 2016 x86_64 x86_64 x86_64 GNU/Linux Build tool maven Personality /testptch/hadoop/patchprocess/precommit/personality/provided.sh git revision HADOOP-13345 / 566c8b7 Default Java 1.8.0_121 compile https://builds.apache.org/job/PreCommit-HADOOP-Build/11824/artifact/patchprocess/branch-compile-root.txt findbugs v3.0.0 compile https://builds.apache.org/job/PreCommit-HADOOP-Build/11824/artifact/patchprocess/patch-compile-root.txt javac https://builds.apache.org/job/PreCommit-HADOOP-Build/11824/artifact/patchprocess/patch-compile-root.txt checkstyle https://builds.apache.org/job/PreCommit-HADOOP-Build/11824/artifact/patchprocess/diff-checkstyle-root.txt mvnsite https://builds.apache.org/job/PreCommit-HADOOP-Build/11824/artifact/patchprocess/patch-mvnsite-hadoop-tools_hadoop-aws.txt whitespace https://builds.apache.org/job/PreCommit-HADOOP-Build/11824/artifact/patchprocess/whitespace-eol.txt javadoc https://builds.apache.org/job/PreCommit-HADOOP-Build/11824/artifact/patchprocess/diff-javadoc-javadoc-hadoop-mapreduce-project_hadoop-mapreduce-client_hadoop-mapreduce-client-core.txt javadoc https://builds.apache.org/job/PreCommit-HADOOP-Build/11824/artifact/patchprocess/patch-javadoc-hadoop-tools_hadoop-aws.txt unit https://builds.apache.org/job/PreCommit-HADOOP-Build/11824/artifact/patchprocess/patch-unit-hadoop-tools_hadoop-aws.txt Test Results https://builds.apache.org/job/PreCommit-HADOOP-Build/11824/testReport/ modules C: hadoop-common-project/hadoop-common hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core hadoop-tools/hadoop-aws U: . Console output https://builds.apache.org/job/PreCommit-HADOOP-Build/11824/console Powered by Apache Yetus 0.5.0-SNAPSHOT http://yetus.apache.org This message was automatically generated.
          Hide
          stevel@apache.org Steve Loughran added a comment - - edited

          Patch 013.

          Becoming ready for review, the merge is done as far as core functionality through a direct s3 connection is concerned. I've migrated Ryan's code and fixed his tests and mine.

          1. Unified the commit logic between staging committers; there's an explicit preCommit(context, pending) in StagingS3GuardCommitter for subclasses to override (Base: no-op); if the precommit fails, pending ops are rolled back (existing code in PartitionStagingCommitter.
          2. added subdirectory support to the DirectoryStagingCommitter
          3. added option to create _SUCCESS markers in these commiters, default is true, as per the FileOutputCommitter
          4. some more tuning in the conflict resolution to trim the number of S3 calls. Every getFileStatus/exists call is sacred
          5. all tests are passing
          6. all integration tests are passing

          TODO

          • More tests needed, obviously. I'm thinking of some scale ones with many files
          • some metrics; it'd be good to know the number of files and amount of bytes uploaded in commits. This is implicitly measured, but not called out. Knowing bytes uploaded in commit will show impact of commit. If compared with files_copied_bytes (done in S3) you can start to estimate impact of operations.
          • switch to FileCommitActions and the S3a methods for the s3 ops. This will require us to mock all that stuff too; still thinking of best way.
          • the MPU commits to update s3guard

          As an aside, because MPUs have request IDs which can be cancelled explicitly, it could be possible for a future committer to actually write direct to S3, saving the pending data to the committer directory in HDFS. This would blur the magic committer with the staging one; just rely on HDFS to implement the consistent commit logic through renames, and so use s3 as a destination with or without consistency there. I think you'd still need the _magic directory though, so the working dir would trigger the special output operation.

          Show
          stevel@apache.org Steve Loughran added a comment - - edited Patch 013. Becoming ready for review, the merge is done as far as core functionality through a direct s3 connection is concerned. I've migrated Ryan's code and fixed his tests and mine. Unified the commit logic between staging committers; there's an explicit preCommit(context, pending) in StagingS3GuardCommitter for subclasses to override (Base: no-op); if the precommit fails, pending ops are rolled back (existing code in PartitionStagingCommitter. added subdirectory support to the DirectoryStagingCommitter added option to create _SUCCESS markers in these commiters, default is true, as per the FileOutputCommitter some more tuning in the conflict resolution to trim the number of S3 calls. Every getFileStatus/exists call is sacred all tests are passing all integration tests are passing TODO More tests needed, obviously. I'm thinking of some scale ones with many files some metrics; it'd be good to know the number of files and amount of bytes uploaded in commits. This is implicitly measured, but not called out. Knowing bytes uploaded in commit will show impact of commit. If compared with files_copied_bytes (done in S3) you can start to estimate impact of operations. switch to FileCommitActions and the S3a methods for the s3 ops. This will require us to mock all that stuff too; still thinking of best way. the MPU commits to update s3guard As an aside, because MPUs have request IDs which can be cancelled explicitly, it could be possible for a future committer to actually write direct to S3, saving the pending data to the committer directory in HDFS. This would blur the magic committer with the staging one; just rely on HDFS to implement the consistent commit logic through renames, and so use s3 as a destination with or without consistency there. I think you'd still need the _magic directory though, so the working dir would trigger the special output operation.
          Hide
          rdblue Ryan Blue added a comment -

          On metrics: our layer on top of the staging committer tracks metrics and sends them to the job committer using the same PendingCommit that already gets serialized. That's an easy way to get more data back to the job committer, which then accumulates the number of files, sizes, etc. and stores it somewhere (or logs it?).

          Show
          rdblue Ryan Blue added a comment - On metrics: our layer on top of the staging committer tracks metrics and sends them to the job committer using the same PendingCommit that already gets serialized. That's an easy way to get more data back to the job committer, which then accumulates the number of files, sizes, etc. and stores it somewhere (or logs it?).
          Hide
          stevel@apache.org Steve Loughran added a comment -

          For metrics, S3a now collects a very detailed set, on low level HTTP verbs, stream reads (including aborts, seek lengths), upload data, errors seen: https://github.com/apache/hadoop/blob/trunk/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Statistic.java

          S3a toString prints them; the input & output streams also print theirs. All 2.8+ filesystems support a new method, {{getStorageStatistics() }} which return the map of name -> counter value; HDFS & s3a implement this & share common keys where possible.

          For a 2.8+ committer only, I plan to grab the FS details from every task and, like you say, copy them back to the job for printing. Spark itself could be extended there; while moving to 2.8+ isn't going to happen, if there was a way for every job to provide a key->long map, that 2.8+ committer could return the stats.

          Side issue: filesystems are per user; multiple executors on the same FS will pick up shared values. We could have the committer independently track the bytes uploaded in the commit, that being the key delay. Acutally, that's implicitly in there in the object length field.

          I like the idea of serializing in the pending commit data, though that doesn't count the details of uncommitted IO, does it. We'd really want that too, somehow. It's still work —just wasted work.

          Tez already grabs and logs the storage stats, uploads them to ATS, incidentally.

          Show
          stevel@apache.org Steve Loughran added a comment - For metrics, S3a now collects a very detailed set, on low level HTTP verbs, stream reads (including aborts, seek lengths), upload data, errors seen: https://github.com/apache/hadoop/blob/trunk/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Statistic.java S3a toString prints them; the input & output streams also print theirs. All 2.8+ filesystems support a new method, {{getStorageStatistics() }} which return the map of name -> counter value; HDFS & s3a implement this & share common keys where possible. For a 2.8+ committer only, I plan to grab the FS details from every task and, like you say, copy them back to the job for printing. Spark itself could be extended there; while moving to 2.8+ isn't going to happen, if there was a way for every job to provide a key->long map, that 2.8+ committer could return the stats. Side issue: filesystems are per user; multiple executors on the same FS will pick up shared values. We could have the committer independently track the bytes uploaded in the commit, that being the key delay. Acutally, that's implicitly in there in the object length field. I like the idea of serializing in the pending commit data, though that doesn't count the details of uncommitted IO, does it. We'd really want that too, somehow. It's still work —just wasted work. Tez already grabs and logs the storage stats, uploads them to ATS, incidentally.
          Hide
          stevel@apache.org Steve Loughran added a comment -

          patch 015 (skipping 14 for various reasons)

          • mostly trying to get checkstyle to STFU, except where I felt it was spurious.
          • IT committer tests verifies that task commit doesn't materialize any part- files in the dest dir. The magic committer does; I've got that bit of the protocol wrong. Which now leaves me with the problem of : how to get the information about committed tasks into the job dir without renames? Plan: read in the .pending files and save a single "all PUTs in this task" file, unified with the staging committers; delete the task area
          Show
          stevel@apache.org Steve Loughran added a comment - patch 015 (skipping 14 for various reasons) mostly trying to get checkstyle to STFU, except where I felt it was spurious. IT committer tests verifies that task commit doesn't materialize any part- files in the dest dir. The magic committer does; I've got that bit of the protocol wrong. Which now leaves me with the problem of : how to get the information about committed tasks into the job dir without renames? Plan: read in the .pending files and save a single "all PUTs in this task" file, unified with the staging committers; delete the task area
          Hide
          stevel@apache.org Steve Loughran added a comment -

          Patch 016, jenkins issues

          • checkstyles, still leaving line length. I really can't be motivated to encapsulate all the fields in test base classes. This aint production code.
          • failing test pair look due to ordering of list output; sorting them both before comparing

          No other changes

          The test run here really played up, including unrelated stuff like ITestS3AContractDistCp, which run standalone. I think there's multipart upload cleanup going on which is interfering with parallel tests.

          Plan:

          1. identify where that's happening.
          2. have a sysprop indicating parallel/serial test running, and in serial mode: cleanup
          Show
          stevel@apache.org Steve Loughran added a comment - Patch 016, jenkins issues checkstyles, still leaving line length. I really can't be motivated to encapsulate all the fields in test base classes. This aint production code. failing test pair look due to ordering of list output; sorting them both before comparing No other changes The test run here really played up, including unrelated stuff like ITestS3AContractDistCp , which run standalone. I think there's multipart upload cleanup going on which is interfering with parallel tests. Plan: identify where that's happening. have a sysprop indicating parallel/serial test running, and in serial mode: cleanup
          Hide
          stevel@apache.org Steve Loughran added a comment -

          patch 017
          *More tests of what happens in various abort sequences; I'll next have the staging task committers provide a way to locate their local dirs, so I can verify they get deleted.

          • Minor doc tuning...turning off the TOC macro gets it to render.
          • new ITest case testAbortJobNotTask shows that Magic committer jobAbort() doesn't reliably abort pending requests from tasks. While I'm focusing on the staging, I do want this to at least be passing the basic tests.
          Show
          stevel@apache.org Steve Loughran added a comment - patch 017 *More tests of what happens in various abort sequences; I'll next have the staging task committers provide a way to locate their local dirs, so I can verify they get deleted. Minor doc tuning...turning off the TOC macro gets it to render. new ITest case testAbortJobNotTask shows that Magic committer jobAbort() doesn't reliably abort pending requests from tasks. While I'm focusing on the staging, I do want this to at least be passing the basic tests.
          Hide
          stevel@apache.org Steve Loughran added a comment -

          Patch 018

          1. Lot more testing on lifecycle corner cases, especially sequences like "commit job while a task was neither committed nor aborted"
          2. tests moving towards supporting part-0000+UUID in path asserts, though not re-enabled the UUID config option yet.
          3. lots more cleanup in committers of: pending commits, local staging dirs.
          4. the staging committer throws an exception if the cluster default FS == s3a, at least for now it requires hdfs, file, etc. I don't want confusion in testing, maybe once we are happy that s3guard delivers the consistency we need then it can be supported.

          The magic committer is failing tests, it's clear its still immature: I'm focusing on the staging ones. The only main changes there are in cleanup/abort logic.

          I've now got this working with Spark; people should look at https://github.com/steveloughran/spark-cloud-examples for some tests there.

          Show
          stevel@apache.org Steve Loughran added a comment - Patch 018 Lot more testing on lifecycle corner cases, especially sequences like "commit job while a task was neither committed nor aborted" tests moving towards supporting part-0000+UUID in path asserts, though not re-enabled the UUID config option yet. lots more cleanup in committers of: pending commits, local staging dirs. the staging committer throws an exception if the cluster default FS == s3a, at least for now it requires hdfs, file, etc. I don't want confusion in testing, maybe once we are happy that s3guard delivers the consistency we need then it can be supported. The magic committer is failing tests, it's clear its still immature: I'm focusing on the staging ones. The only main changes there are in cleanup/abort logic. I've now got this working with Spark; people should look at https://github.com/steveloughran/spark-cloud-examples for some tests there.
          Hide
          fabbri Aaron Fabbri added a comment -

          I'll try to get you some useful code reviews soon.. Been busy with releases. Wow there is a lot of code to catch up on here.

          Show
          fabbri Aaron Fabbri added a comment - I'll try to get you some useful code reviews soon.. Been busy with releases. Wow there is a lot of code to catch up on here.
          Hide
          stevel@apache.org Steve Loughran added a comment -

          Its not so much the code quantity as the fact you end up with having to know the commit protocol before you can judge the quality of it, and that's something else to learn. I learned it by single-stepping through the debugger.

          I think I could/should split out the magic committer as it is the one doing more to the internals of s3aFS. I think it's a good evolutionary feature, but the staging one should ship first. In which case some of the S3aFS logic can be temporarily culled, though I'd like to retain the changes to the block output stream to allow it to write-complete vs write-pending. the enhancements to WriteOperationsHelper, json persistent format &c are actually things the staging committer(s) can share -and will be needed to keep s3guard's DDB tables up to speed.

          I could to a tour early next week, say tuesday morning PST?

          Show
          stevel@apache.org Steve Loughran added a comment - Its not so much the code quantity as the fact you end up with having to know the commit protocol before you can judge the quality of it, and that's something else to learn. I learned it by single-stepping through the debugger. I think I could/should split out the magic committer as it is the one doing more to the internals of s3aFS. I think it's a good evolutionary feature, but the staging one should ship first. In which case some of the S3aFS logic can be temporarily culled, though I'd like to retain the changes to the block output stream to allow it to write-complete vs write-pending. the enhancements to WriteOperationsHelper, json persistent format &c are actually things the staging committer(s) can share -and will be needed to keep s3guard's DDB tables up to speed. I could to a tour early next week, say tuesday morning PST?
          Hide
          fabbri Aaron Fabbri added a comment -

          you end up with having to know the commit protocol before you can judge the quality of it, and that's something else to learn.

          Indeed. Also, I appreciated the documentation included in the patch, thanks.

          I could to a tour early next week, say tuesday morning PST?

          Yes, that would be great.

          Show
          fabbri Aaron Fabbri added a comment - you end up with having to know the commit protocol before you can judge the quality of it, and that's something else to learn. Indeed. Also, I appreciated the documentation included in the patch, thanks. I could to a tour early next week, say tuesday morning PST? Yes, that would be great.
          Hide
          Thomas Demoor Thomas Demoor added a comment -

          On the staging wrappedCommitter exception on s3a. Could we have a config flag to override that? I think that would satisfy test sanity but still enable consistent s3 implementations to run without deploying an hdfs cluster just for this.

          I'd be interested in a tour, in my first review I quickly found out that the rabbithole is deep

          Show
          Thomas Demoor Thomas Demoor added a comment - On the staging wrappedCommitter exception on s3a. Could we have a config flag to override that? I think that would satisfy test sanity but still enable consistent s3 implementations to run without deploying an hdfs cluster just for this. I'd be interested in a tour, in my first review I quickly found out that the rabbithole is deep
          Hide
          stevel@apache.org Steve Loughran added a comment -

          Thomas, I'm aware of your needs. I just put that lock in there for now as my integration tests did have s3a as the default FS, and the debug logs had me confused. Will make it an option.

          How would a webex at 09:00 PST work next week; the EU will have switched TZ so it'll be 17:00 my time,, 18:00 thomas's. Or I can slip to 10:00 PST if that suits

          Show
          stevel@apache.org Steve Loughran added a comment - Thomas, I'm aware of your needs. I just put that lock in there for now as my integration tests did have s3a as the default FS, and the debug logs had me confused. Will make it an option. How would a webex at 09:00 PST work next week; the EU will have switched TZ so it'll be 17:00 my time,, 18:00 thomas's. Or I can slip to 10:00 PST if that suits
          Hide
          fabbri Aaron Fabbri added a comment -

          How would a webex at 09:00 PST work next week

          Works for me.. 10 PST is good as well.

          Show
          fabbri Aaron Fabbri added a comment - How would a webex at 09:00 PST work next week Works for me.. 10 PST is good as well.
          Hide
          stevel@apache.org Steve Loughran added a comment -

          HADOOP-13786 patch 019: cleanup of jobs & tasks

          • Lots of work on cleanup, including abortjob purging all pending uploads against a dest. Without that, there's actually a problem in staging aborts. But what? testRecoveryAndCleanup was finding the problem
          • this is starting to use commitActions in Staging tests, complicating mocking a bit (more methods need to be handled/stubbed).
          • a bit more docs on lifecycle, but not enough.
          • still confusion about when to use (deprecated) cleanup() calls vs abort?
          • track "role" of an instance, included in logs and exceptions. This makes a big difference in IT tests downstream where there is >1 role/instance per process and you want to identify which is doing work.

          Sort term todos:

          • IT test on commit contention, verify that the committed output completes
          • an IT test of a real MR job
          • more testing in my downstream module
          • better specification of what's going on with the commit protocol
          Show
          stevel@apache.org Steve Loughran added a comment - HADOOP-13786 patch 019: cleanup of jobs & tasks Lots of work on cleanup, including abortjob purging all pending uploads against a dest. Without that, there's actually a problem in staging aborts. But what? testRecoveryAndCleanup was finding the problem this is starting to use commitActions in Staging tests, complicating mocking a bit (more methods need to be handled/stubbed). a bit more docs on lifecycle, but not enough. still confusion about when to use (deprecated) cleanup() calls vs abort? track "role" of an instance, included in logs and exceptions. This makes a big difference in IT tests downstream where there is >1 role/instance per process and you want to identify which is doing work. Sort term todos: IT test on commit contention, verify that the committed output completes an IT test of a real MR job more testing in my downstream module better specification of what's going on with the commit protocol
          Hide
          stevel@apache.org Steve Loughran added a comment -

          And for the very curious, here's a trace of a test run of the latest code doing some simple sparkContact.makeRDD(1 to count).saveAsTextFile("s3a:///something") with the directory committer. There's a job committer and task committer in the logs, hence the role log info to work out whats happening. fs.s3a mostly at debug, except for instrumentation. FS stats exclude all the info on the uploads because they (currently) go straight through the s3a client.

              
          -------------------------------------------
          2017-03-23 18:42:41,455 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] DEBUG s3a.S3AFileSystem (S3AFileSystem.java:innerGetFileStatus(1782)) - Getting path status for s3a://hwdev-steve-new/spark-cloud/S3ANumbersSuiteV2APISuite/numbers_rdd_tests_v2api  (spark-cloud/S3ANumbersSuiteV2APISuite/numbers_rdd_tests_v2api)
          2017-03-23 18:42:42,024 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] DEBUG s3a.S3AFileSystem (S3AFileSystem.java:s3GetFileStatus(1905)) - Not Found: s3a://hwdev-steve-new/spark-cloud/S3ANumbersSuiteV2APISuite/numbers_rdd_tests_v2api
          2017-03-23 18:42:42,024 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] DEBUG s3a.S3AFileSystem (S3AFileSystem.java:delete(1418)) - Couldn't delete s3a://hwdev-steve-new/spark-cloud/S3ANumbersSuiteV2APISuite/numbers_rdd_tests_v2api - does not exist
          2017-03-23 18:42:42,024 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] INFO  s3.S3ANumbersSuiteV2APISuite (CloudLogging.scala:logInfo(56)) - Switching to local file:// fs for default FS
          2017-03-23 18:42:42,098 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] INFO  spark.SparkContext (Logging.scala:logInfo(54)) - Running Spark version 2.2.0-SNAPSHOT
          2017-03-23 18:42:42,133 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] INFO  spark.SparkContext (Logging.scala:logInfo(54)) - Submitted application: test
          2017-03-23 18:42:42,147 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] INFO  spark.SecurityManager (Logging.scala:logInfo(54)) - Changing view acls to: stevel
          2017-03-23 18:42:42,147 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] INFO  spark.SecurityManager (Logging.scala:logInfo(54)) - Changing modify acls to: stevel
          2017-03-23 18:42:42,148 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] INFO  spark.SecurityManager (Logging.scala:logInfo(54)) - Changing view acls groups to: 
          2017-03-23 18:42:42,149 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] INFO  spark.SecurityManager (Logging.scala:logInfo(54)) - Changing modify acls groups to: 
          2017-03-23 18:42:42,150 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] INFO  spark.SecurityManager (Logging.scala:logInfo(54)) - SecurityManager: authentication disabled; ui acls disabled; users  with view permissions: Set(stevel); groups with view permissions: Set(); users  with modify permissions: Set(stevel); groups with modify permissions: Set()
          2017-03-23 18:42:42,369 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] INFO  util.Utils (Logging.scala:logInfo(54)) - Successfully started service 'sparkDriver' on port 54392.
          2017-03-23 18:42:42,389 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] INFO  spark.SparkEnv (Logging.scala:logInfo(54)) - Registering MapOutputTracker
          2017-03-23 18:42:42,402 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] INFO  spark.SparkEnv (Logging.scala:logInfo(54)) - Registering BlockManagerMaster
          2017-03-23 18:42:42,405 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] INFO  storage.BlockManagerMasterEndpoint (Logging.scala:logInfo(54)) - Using org.apache.spark.storage.DefaultTopologyMapper for getting topology information
          2017-03-23 18:42:42,405 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] INFO  storage.BlockManagerMasterEndpoint (Logging.scala:logInfo(54)) - BlockManagerMasterEndpoint up
          2017-03-23 18:42:42,412 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] INFO  storage.DiskBlockManager (Logging.scala:logInfo(54)) - Created local directory at /Users/stevel/Projects/sparkwork/spark-cloud-examples/cloud-examples/target/tmp/blockmgr-9ab575c8-964f-4cb7-96cb-e5c6efc04d60
          2017-03-23 18:42:42,429 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] INFO  memory.MemoryStore (Logging.scala:logInfo(54)) - MemoryStore started with capacity 2004.6 MB
          2017-03-23 18:42:42,466 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] INFO  spark.SparkEnv (Logging.scala:logInfo(54)) - Registering OutputCommitCoordinator
          2017-03-23 18:42:42,579 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] INFO  util.Utils (Logging.scala:logInfo(54)) - Successfully started service 'SparkUI' on port 4040.
          2017-03-23 18:42:42,606 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] INFO  ui.SparkUI (Logging.scala:logInfo(54)) - Bound SparkUI to 0.0.0.0, and started at http://192.168.1.38:4040
          2017-03-23 18:42:42,679 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] INFO  executor.Executor (Logging.scala:logInfo(54)) - Starting executor ID driver on host localhost
          2017-03-23 18:42:42,694 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] INFO  util.Utils (Logging.scala:logInfo(54)) - Successfully started service 'org.apache.spark.network.netty.NettyBlockTransferService' on port 54394.
          2017-03-23 18:42:42,695 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] INFO  netty.NettyBlockTransferService (Logging.scala:logInfo(54)) - Server created on 192.168.1.38:54394
          2017-03-23 18:42:42,696 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] INFO  storage.BlockManager (Logging.scala:logInfo(54)) - Using org.apache.spark.storage.RandomBlockReplicationPolicy for block replication policy
          2017-03-23 18:42:42,697 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] INFO  storage.BlockManagerMaster (Logging.scala:logInfo(54)) - Registering BlockManager BlockManagerId(driver, 192.168.1.38, 54394, None)
          2017-03-23 18:42:42,701 [dispatcher-event-loop-2] INFO  storage.BlockManagerMasterEndpoint (Logging.scala:logInfo(54)) - Registering block manager 192.168.1.38:54394 with 2004.6 MB RAM, BlockManagerId(driver, 192.168.1.38, 54394, None)
          2017-03-23 18:42:42,703 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] INFO  storage.BlockManagerMaster (Logging.scala:logInfo(54)) - Registered BlockManager BlockManagerId(driver, 192.168.1.38, 54394, None)
          2017-03-23 18:42:42,703 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] INFO  storage.BlockManager (Logging.scala:logInfo(54)) - Initialized BlockManager: BlockManagerId(driver, 192.168.1.38, 54394, None)
          2017-03-23 18:42:42,898 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] INFO  s3.S3ANumbersSuiteV2APISuite (CloudLogging.scala:logInfo(56)) - 
          Generating output under s3a://hwdev-steve-new/spark-cloud/S3ANumbersSuiteV2APISuite/numbers_rdd_tests_v2api
          
          2017-03-23 18:42:42,940 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] DEBUG s3a.S3AFileSystem (S3AFileSystem.java:innerGetFileStatus(1782)) - Getting path status for s3a://hwdev-steve-new/spark-cloud/S3ANumbersSuiteV2APISuite/numbers_rdd_tests_v2api  (spark-cloud/S3ANumbersSuiteV2APISuite/numbers_rdd_tests_v2api)
          2017-03-23 18:42:43,486 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] DEBUG s3a.S3AFileSystem (S3AFileSystem.java:s3GetFileStatus(1905)) - Not Found: s3a://hwdev-steve-new/spark-cloud/S3ANumbersSuiteV2APISuite/numbers_rdd_tests_v2api
          2017-03-23 18:42:43,492 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] DEBUG output.PathOutputCommitterFactory (PathOutputCommitterFactory.java:getOutputCommitterFactory(92)) - Using FileOutputCommitter factory class class org.apache.hadoop.fs.s3a.commit.staging.DirectoryStagingCommitterFactory
          2017-03-23 18:42:43,495 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] DEBUG commit.AbstractS3GuardCommitter (AbstractS3GuardCommitter.java:<init>(85)) - Task committer attempt_20170323184243_0000_m_000000_0 instantiated for job "" ID job_20170323184243_0000 with destination s3a://hwdev-steve-new/spark-cloud/S3ANumbersSuiteV2APISuite/numbers_rdd_tests_v2api
          2017-03-23 18:42:43,497 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] DEBUG commit.AbstractS3GuardCommitter (AbstractS3GuardCommitter.java:<init>(115)) - Task committer attempt_20170323184243_0000_m_000000_0} instantiated for"" ID job_20170323184243_0000
          2017-03-23 18:42:43,499 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] DEBUG commit.AbstractS3GuardCommitter (AbstractS3GuardCommitter.java:setWorkPath(162)) - Setting work path to file:/Users/stevel/Projects/sparkwork/spark-cloud-examples/cloud-examples/target/test/mapred/local/job_20170323184243_0000/0/attempt_20170323184243_0000_m_000000_0/_temporary/0/_temporary/attempt_20170323184243_0000_m_000000_0
          2017-03-23 18:42:43,501 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] INFO  output.FileOutputCommitter (FileOutputCommitter.java:<init>(123)) - File Output Committer Algorithm version is 1
          2017-03-23 18:42:43,501 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] INFO  output.FileOutputCommitter (FileOutputCommitter.java:<init>(138)) - FileOutputCommitter skip cleanup _temporary folders under output directory:false, ignore cleanup failures: true
          2017-03-23 18:42:43,502 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] DEBUG staging.StagingS3GuardCommitter (StagingS3GuardCommitter.java:postCreationActions(168)) - Conflict resolution mode: FAIL
          2017-03-23 18:42:43,502 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] INFO  commit.Abstract3GuardCommitterFactory (Abstract3GuardCommitterFactory.java:createOutputCommitter(61)) - Using Commmitter StagingS3GuardCommitter{AbstractS3GuardCommitter{role=Task committer attempt_20170323184243_0000_m_000000_0outputPath=s3a://hwdev-steve-new/spark-cloud/S3ANumbersSuiteV2APISuite/numbers_rdd_tests_v2api, workPath=file:/Users/stevel/Projects/sparkwork/spark-cloud-examples/cloud-examples/target/test/mapred/local/job_20170323184243_0000/0/attempt_20170323184243_0000_m_000000_0/_temporary/0/_temporary/attempt_20170323184243_0000_m_000000_0}, finalOutputPath=null, conflictResolution=FAIL, wrappedCommitter=FileOutputCommitter{outputPath=file:/Users/stevel/Projects/sparkwork/spark-cloud-examples/cloud-examples/target/tmp/stevel/job_20170323184243_0000/staging-uploads, workPath=null, algorithmVersion=1, skipCleanup=false, ignoreCleanupFailures=true}} for s3a://hwdev-steve-new/spark-cloud/S3ANumbersSuiteV2APISuite/numbers_rdd_tests_v2api
          2017-03-23 18:42:43,502 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] DEBUG staging.StagingS3GuardCommitter (StagingS3GuardCommitter.java:setupJob(520)) - Task committer attempt_20170323184243_0000_m_000000_0, Setting up job job_20170323184242_0001
          2017-03-23 18:42:43,539 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] DEBUG staging.StagingS3GuardCommitter (StagingS3GuardCommitter.java:getOutputPath(1057)) - Task committer attempt_20170323184243_0000_m_000000_0: final output path is s3a://hwdev-steve-new/spark-cloud/S3ANumbersSuiteV2APISuite/numbers_rdd_tests_v2api
          2017-03-23 18:42:43,539 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] DEBUG s3a.S3AFileSystem (S3AFileSystem.java:innerGetFileStatus(1782)) - Getting path status for s3a://hwdev-steve-new/spark-cloud/S3ANumbersSuiteV2APISuite/numbers_rdd_tests_v2api  (spark-cloud/S3ANumbersSuiteV2APISuite/numbers_rdd_tests_v2api)
          2017-03-23 18:42:44,084 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] DEBUG s3a.S3AFileSystem (S3AFileSystem.java:s3GetFileStatus(1905)) - Not Found: s3a://hwdev-steve-new/spark-cloud/S3ANumbersSuiteV2APISuite/numbers_rdd_tests_v2api
          2017-03-23 18:42:44,122 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] INFO  spark.SparkContext (Logging.scala:logInfo(54)) - Starting job: runJob at SparkHadoopMapReduceWriter.scala:88
          2017-03-23 18:42:44,131 [dag-scheduler-event-loop] INFO  scheduler.DAGScheduler (Logging.scala:logInfo(54)) - Got job 0 (runJob at SparkHadoopMapReduceWriter.scala:88) with 1 output partitions
          2017-03-23 18:42:44,132 [dag-scheduler-event-loop] INFO  scheduler.DAGScheduler (Logging.scala:logInfo(54)) - Final stage: ResultStage 0 (runJob at SparkHadoopMapReduceWriter.scala:88)
          2017-03-23 18:42:44,132 [dag-scheduler-event-loop] INFO  scheduler.DAGScheduler (Logging.scala:logInfo(54)) - Parents of final stage: List()
          2017-03-23 18:42:44,133 [dag-scheduler-event-loop] INFO  scheduler.DAGScheduler (Logging.scala:logInfo(54)) - Missing parents: List()
          2017-03-23 18:42:44,137 [dag-scheduler-event-loop] INFO  scheduler.DAGScheduler (Logging.scala:logInfo(54)) - Submitting ResultStage 0 (MapPartitionsRDD[1] at map at NumbersRddTests.scala:120), which has no missing parents
          2017-03-23 18:42:44,208 [dag-scheduler-event-loop] INFO  memory.MemoryStore (Logging.scala:logInfo(54)) - Block broadcast_0 stored as values in memory (estimated size 74.9 KB, free 2004.5 MB)
          2017-03-23 18:42:44,232 [dag-scheduler-event-loop] INFO  memory.MemoryStore (Logging.scala:logInfo(54)) - Block broadcast_0_piece0 stored as bytes in memory (estimated size 25.0 KB, free 2004.5 MB)
          2017-03-23 18:42:44,234 [dispatcher-event-loop-4] INFO  storage.BlockManagerInfo (Logging.scala:logInfo(54)) - Added broadcast_0_piece0 in memory on 192.168.1.38:54394 (size: 25.0 KB, free: 2004.6 MB)
          2017-03-23 18:42:44,236 [dag-scheduler-event-loop] INFO  spark.SparkContext (Logging.scala:logInfo(54)) - Created broadcast 0 from broadcast at DAGScheduler.scala:1002
          2017-03-23 18:42:44,251 [dag-scheduler-event-loop] INFO  scheduler.DAGScheduler (Logging.scala:logInfo(54)) - Submitting 1 missing tasks from ResultStage 0 (MapPartitionsRDD[1] at map at NumbersRddTests.scala:120) (first 15 tasks are for partitions Vector(0))
          2017-03-23 18:42:44,252 [dag-scheduler-event-loop] INFO  scheduler.TaskSchedulerImpl (Logging.scala:logInfo(54)) - Adding task set 0.0 with 1 tasks
          2017-03-23 18:42:44,281 [dispatcher-event-loop-5] INFO  scheduler.TaskSetManager (Logging.scala:logInfo(54)) - Starting task 0.0 in stage 0.0 (TID 0, localhost, executor driver, partition 0, PROCESS_LOCAL, 7598 bytes)
          2017-03-23 18:42:44,286 [Executor task launch worker for task 0] INFO  executor.Executor (Logging.scala:logInfo(54)) - Running task 0.0 in stage 0.0 (TID 0)
          2017-03-23 18:42:44,333 [Executor task launch worker for task 0] DEBUG output.PathOutputCommitterFactory (PathOutputCommitterFactory.java:getOutputCommitterFactory(92)) - Using FileOutputCommitter factory class class org.apache.hadoop.fs.s3a.commit.staging.DirectoryStagingCommitterFactory
          2017-03-23 18:42:44,334 [Executor task launch worker for task 0] DEBUG commit.AbstractS3GuardCommitter (AbstractS3GuardCommitter.java:<init>(85)) - Task committer attempt_20170323184242_0000_r_000000_0 instantiated for job "" ID job_20170323184242_0000 with destination s3a://hwdev-steve-new/spark-cloud/S3ANumbersSuiteV2APISuite/numbers_rdd_tests_v2api
          2017-03-23 18:42:44,335 [Executor task launch worker for task 0] DEBUG commit.AbstractS3GuardCommitter (AbstractS3GuardCommitter.java:<init>(115)) - Task committer attempt_20170323184242_0000_r_000000_0} instantiated for"" ID job_20170323184242_0000
          2017-03-23 18:42:44,336 [Executor task launch worker for task 0] DEBUG commit.AbstractS3GuardCommitter (AbstractS3GuardCommitter.java:setWorkPath(162)) - Setting work path to file:/Users/stevel/Projects/sparkwork/spark-cloud-examples/cloud-examples/target/test/mapred/local/job_20170323184243_0000/0/attempt_20170323184242_0000_r_000000_0/_temporary/0/_temporary/attempt_20170323184242_0000_r_000000_0
          2017-03-23 18:42:44,337 [Executor task launch worker for task 0] INFO  output.FileOutputCommitter (FileOutputCommitter.java:<init>(123)) - File Output Committer Algorithm version is 1
          2017-03-23 18:42:44,337 [Executor task launch worker for task 0] INFO  output.FileOutputCommitter (FileOutputCommitter.java:<init>(138)) - FileOutputCommitter skip cleanup _temporary folders under output directory:false, ignore cleanup failures: true
          2017-03-23 18:42:44,337 [Executor task launch worker for task 0] DEBUG staging.StagingS3GuardCommitter (StagingS3GuardCommitter.java:postCreationActions(168)) - Conflict resolution mode: FAIL
          2017-03-23 18:42:44,338 [Executor task launch worker for task 0] INFO  commit.Abstract3GuardCommitterFactory (Abstract3GuardCommitterFactory.java:createOutputCommitter(61)) - Using Commmitter StagingS3GuardCommitter{AbstractS3GuardCommitter{role=Task committer attempt_20170323184242_0000_r_000000_0outputPath=s3a://hwdev-steve-new/spark-cloud/S3ANumbersSuiteV2APISuite/numbers_rdd_tests_v2api, workPath=file:/Users/stevel/Projects/sparkwork/spark-cloud-examples/cloud-examples/target/test/mapred/local/job_20170323184243_0000/0/attempt_20170323184242_0000_r_000000_0/_temporary/0/_temporary/attempt_20170323184242_0000_r_000000_0}, finalOutputPath=null, conflictResolution=FAIL, wrappedCommitter=FileOutputCommitter{outputPath=file:/Users/stevel/Projects/sparkwork/spark-cloud-examples/cloud-examples/target/tmp/stevel/job_20170323184243_0000/staging-uploads, workPath=null, algorithmVersion=1, skipCleanup=false, ignoreCleanupFailures=true}} for s3a://hwdev-steve-new/spark-cloud/S3ANumbersSuiteV2APISuite/numbers_rdd_tests_v2api
          2017-03-23 18:42:44,339 [Executor task launch worker for task 0] INFO  commit.DurationInfo (DurationInfo.java:<init>(33)) - Starting Task committer attempt_20170323184242_0000_r_000000_0: task attempt_20170323184242_0000_r_000000_0: creating task attempt path file:/Users/stevel/Projects/sparkwork/spark-cloud-examples/cloud-examples/target/test/mapred/local/job_20170323184243_0000/0/attempt_20170323184242_0000_r_000000_0/_temporary/0/_temporary/attempt_20170323184242_0000_r_000000_0 
          2017-03-23 18:42:44,386 [Executor task launch worker for task 0] INFO  commit.DurationInfo (DurationInfo.java:close(44)) - Task committer attempt_20170323184242_0000_r_000000_0: task attempt_20170323184242_0000_r_000000_0: creating task attempt path file:/Users/stevel/Projects/sparkwork/spark-cloud-examples/cloud-examples/target/test/mapred/local/job_20170323184243_0000/0/attempt_20170323184242_0000_r_000000_0/_temporary/0/_temporary/attempt_20170323184242_0000_r_000000_0 : 0:00:047
          2017-03-23 18:42:44,388 [Executor task launch worker for task 0] DEBUG output.PathOutputCommitterFactory (PathOutputCommitterFactory.java:getOutputCommitterFactory(92)) - Using FileOutputCommitter factory class class org.apache.hadoop.fs.s3a.commit.staging.DirectoryStagingCommitterFactory
          2017-03-23 18:42:44,388 [Executor task launch worker for task 0] DEBUG commit.AbstractS3GuardCommitter (AbstractS3GuardCommitter.java:<init>(85)) - Task committer attempt_20170323184242_0000_r_000000_0 instantiated for job "" ID job_20170323184242_0000 with destination s3a://hwdev-steve-new/spark-cloud/S3ANumbersSuiteV2APISuite/numbers_rdd_tests_v2api
          2017-03-23 18:42:44,388 [Executor task launch worker for task 0] DEBUG commit.AbstractS3GuardCommitter (AbstractS3GuardCommitter.java:<init>(115)) - Task committer attempt_20170323184242_0000_r_000000_0} instantiated for"" ID job_20170323184242_0000
          2017-03-23 18:42:44,389 [Executor task launch worker for task 0] DEBUG commit.AbstractS3GuardCommitter (AbstractS3GuardCommitter.java:setWorkPath(162)) - Setting work path to file:/Users/stevel/Projects/sparkwork/spark-cloud-examples/cloud-examples/target/test/mapred/local/job_20170323184243_0000/0/attempt_20170323184242_0000_r_000000_0/_temporary/0/_temporary/attempt_20170323184242_0000_r_000000_0
          2017-03-23 18:42:44,390 [Executor task launch worker for task 0] INFO  output.FileOutputCommitter (FileOutputCommitter.java:<init>(123)) - File Output Committer Algorithm version is 1
          2017-03-23 18:42:44,390 [Executor task launch worker for task 0] INFO  output.FileOutputCommitter (FileOutputCommitter.java:<init>(138)) - FileOutputCommitter skip cleanup _temporary folders under output directory:false, ignore cleanup failures: true
          2017-03-23 18:42:44,390 [Executor task launch worker for task 0] DEBUG staging.StagingS3GuardCommitter (StagingS3GuardCommitter.java:postCreationActions(168)) - Conflict resolution mode: FAIL
          2017-03-23 18:42:44,390 [Executor task launch worker for task 0] INFO  commit.Abstract3GuardCommitterFactory (Abstract3GuardCommitterFactory.java:createOutputCommitter(61)) - Using Commmitter StagingS3GuardCommitter{AbstractS3GuardCommitter{role=Task committer attempt_20170323184242_0000_r_000000_0outputPath=s3a://hwdev-steve-new/spark-cloud/S3ANumbersSuiteV2APISuite/numbers_rdd_tests_v2api, workPath=file:/Users/stevel/Projects/sparkwork/spark-cloud-examples/cloud-examples/target/test/mapred/local/job_20170323184243_0000/0/attempt_20170323184242_0000_r_000000_0/_temporary/0/_temporary/attempt_20170323184242_0000_r_000000_0}, finalOutputPath=null, conflictResolution=FAIL, wrappedCommitter=FileOutputCommitter{outputPath=file:/Users/stevel/Projects/sparkwork/spark-cloud-examples/cloud-examples/target/tmp/stevel/job_20170323184243_0000/staging-uploads, workPath=null, algorithmVersion=1, skipCleanup=false, ignoreCleanupFailures=true}} for s3a://hwdev-steve-new/spark-cloud/S3ANumbersSuiteV2APISuite/numbers_rdd_tests_v2api
          2017-03-23 18:42:44,448 [Executor task launch worker for task 0] INFO  commit.DurationInfo (DurationInfo.java:<init>(33)) - Starting Task committer attempt_20170323184242_0000_r_000000_0: needsTaskCommit() Task attempt_20170323184242_0000_r_000000_0
          2017-03-23 18:42:44,454 [Executor task launch worker for task 0] DEBUG staging.StagingS3GuardCommitter (StagingS3GuardCommitter.java:needsTaskCommit(849)) - 1 files to commit
          2017-03-23 18:42:44,455 [Executor task launch worker for task 0] INFO  commit.DurationInfo (DurationInfo.java:close(44)) - Task committer attempt_20170323184242_0000_r_000000_0: needsTaskCommit() Task attempt_20170323184242_0000_r_000000_0: 0:00:006
          2017-03-23 18:42:44,457 [Executor task launch worker for task 0] INFO  commit.DurationInfo (DurationInfo.java:<init>(33)) - Starting Task committer attempt_20170323184242_0000_r_000000_0: commit task attempt_20170323184242_0000_r_000000_0
          2017-03-23 18:42:44,457 [Executor task launch worker for task 0] DEBUG staging.StagingS3GuardCommitter (StagingS3GuardCommitter.java:getTaskOutput(419)) - Scanning file:/Users/stevel/Projects/sparkwork/spark-cloud-examples/cloud-examples/target/test/mapred/local/job_20170323184243_0000/0/attempt_20170323184242_0000_r_000000_0/_temporary/0/_temporary/attempt_20170323184242_0000_r_000000_0 for files to commit
          2017-03-23 18:42:44,468 [Executor task launch worker for task 0] DEBUG staging.StagingS3GuardCommitter (StagingS3GuardCommitter.java:commitTaskInternal(885)) - Task committer attempt_20170323184242_0000_r_000000_0: commitTaskInternal
          2017-03-23 18:42:44,468 [Executor task launch worker for task 0] DEBUG staging.StagingS3GuardCommitter (StagingS3GuardCommitter.java:commitTaskInternal(890)) - Task committer attempt_20170323184242_0000_r_000000_0: attempt path is file:/Users/stevel/Projects/sparkwork/spark-cloud-examples/cloud-examples/target/test/mapred/local/job_20170323184243_0000/0/attempt_20170323184242_0000_r_000000_0/_temporary/0/_temporary/attempt_20170323184242_0000_r_000000_0
          2017-03-23 18:42:44,468 [Executor task launch worker for task 0] INFO  staging.StagingS3GuardCommitter (StagingS3GuardCommitter.java:commitTaskInternal(903)) - Task committer attempt_20170323184242_0000_r_000000_0: uploading from staging directory to S3
          2017-03-23 18:42:44,468 [Executor task launch worker for task 0] INFO  staging.StagingS3GuardCommitter (StagingS3GuardCommitter.java:commitTaskInternal(904)) - Task committer attempt_20170323184242_0000_r_000000_0: Saving pending data information to file:/Users/stevel/Projects/sparkwork/spark-cloud-examples/cloud-examples/target/tmp/stevel/job_20170323184243_0000/staging-uploads/_temporary/0/_temporary/attempt_20170323184242_0000_r_000000_0
          2017-03-23 18:42:44,469 [Executor task launch worker for task 0] DEBUG staging.StagingS3GuardCommitter (StagingS3GuardCommitter.java:getOutputPath(1057)) - Task committer attempt_20170323184242_0000_r_000000_0: final output path is s3a://hwdev-steve-new/spark-cloud/S3ANumbersSuiteV2APISuite/numbers_rdd_tests_v2api
          2017-03-23 18:42:44,493 [Executor task launch worker for task 0] DEBUG staging.S3Util (S3Util.java:multipartUpload(155)) - Initiating multipart upload from /Users/stevel/Projects/sparkwork/spark-cloud-examples/cloud-examples/target/test/mapred/local/job_20170323184243_0000/0/attempt_20170323184242_0000_r_000000_0/_temporary/0/_temporary/attempt_20170323184242_0000_r_000000_0/part-r-00000 to s3a://hwdev-steve-new/spark-cloud/S3ANumbersSuiteV2APISuite/numbers_rdd_tests_v2api/part-r-00000-job_20170323184243_0000 partition=null
          2017-03-23 18:42:44,684 [Executor task launch worker for task 0] DEBUG staging.S3Util (S3Util.java:multipartUpload(177)) - File size is 515303, number of parts to upload = 1
          2017-03-23 18:42:46,352 [Executor task launch worker for task 0] DEBUG staging.StagingS3GuardCommitter (StagingS3GuardCommitter.java:run(939)) - Task committer attempt_20170323184242_0000_r_000000_0: adding pending commit PendingUpload{ to 's3a://hwdev-steve-new/spark-cloud/S3ANumbersSuiteV2APISuite/numbers_rdd_tests_v2api/part-r-00000-job_20170323184243_0000' , uploadId='_P7n0N2FxTdVYiTBBFNB2Y9TOQgnbuKT1Lt.mwfx4pdmT6aK4E5XlxfpFZNwSUv0viLzNx45qvhJAqDyhSDqajozcfuFesRF2enq.bpkfc0-'}
          2017-03-23 18:42:46,360 [Executor task launch worker for task 0] INFO  output.FileOutputCommitter (FileOutputCommitter.java:commitTask(582)) - Saved output of task 'attempt_20170323184242_0000_r_000000_0' to file:/Users/stevel/Projects/sparkwork/spark-cloud-examples/cloud-examples/target/tmp/stevel/job_20170323184243_0000/staging-uploads/_temporary/0/task_20170323184242_0000_r_000000
          2017-03-23 18:42:46,362 [Executor task launch worker for task 0] INFO  staging.StagingS3GuardCommitter (StagingS3GuardCommitter.java:commitTask(866)) - Task committer attempt_20170323184242_0000_r_000000_0: upload file count: 1
          2017-03-23 18:42:46,362 [Executor task launch worker for task 0] INFO  commit.DurationInfo (DurationInfo.java:close(44)) - Task committer attempt_20170323184242_0000_r_000000_0: commit task attempt_20170323184242_0000_r_000000_0: 0:01:906
          2017-03-23 18:42:46,363 [Executor task launch worker for task 0] INFO  mapred.SparkHadoopMapRedUtil (Logging.scala:logInfo(54)) - attempt_20170323184242_0000_r_000000_0: Committed
          2017-03-23 18:42:46,374 [Executor task launch worker for task 0] INFO  executor.Executor (Logging.scala:logInfo(54)) - Finished task 0.0 in stage 0.0 (TID 0). 1306 bytes result sent to driver
          2017-03-23 18:42:46,380 [task-result-getter-0] INFO  scheduler.TaskSetManager (Logging.scala:logInfo(54)) - Finished task 0.0 in stage 0.0 (TID 0) in 2111 ms on localhost (executor driver) (1/1)
          2017-03-23 18:42:46,381 [task-result-getter-0] INFO  scheduler.TaskSchedulerImpl (Logging.scala:logInfo(54)) - Removed TaskSet 0.0, whose tasks have all completed, from pool 
          2017-03-23 18:42:46,384 [dag-scheduler-event-loop] INFO  scheduler.DAGScheduler (Logging.scala:logInfo(54)) - ResultStage 0 (runJob at SparkHadoopMapReduceWriter.scala:88) finished in 2.124 s
          2017-03-23 18:42:46,388 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] INFO  scheduler.DAGScheduler (Logging.scala:logInfo(54)) - Job 0 finished: runJob at SparkHadoopMapReduceWriter.scala:88, took 2.265104 s
          2017-03-23 18:42:46,389 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] INFO  commit.DurationInfo (DurationInfo.java:<init>(33)) - Starting Task committer attempt_20170323184243_0000_m_000000_0: preparing to commit Job
          2017-03-23 18:42:46,390 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] DEBUG staging.StagingS3GuardCommitter (StagingS3GuardCommitter.java:getThreadPool(1094)) - Task committer attempt_20170323184243_0000_m_000000_0: creating thread pool of size 8
          2017-03-23 18:42:46,393 [s3-committer-pool-0] DEBUG staging.S3Util (S3Util.java:readPendingCommits(234)) - Reading pending commits in file file:/Users/stevel/Projects/sparkwork/spark-cloud-examples/cloud-examples/target/tmp/stevel/job_20170323184243_0000/staging-uploads/_temporary/0/task_20170323184242_0000_r_000000
          2017-03-23 18:42:46,406 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] DEBUG s3a.S3AFileSystem (S3AFileSystem.java:innerGetFileStatus(1782)) - Getting path status for s3a://hwdev-steve-new/spark-cloud/S3ANumbersSuiteV2APISuite/numbers_rdd_tests_v2api  (spark-cloud/S3ANumbersSuiteV2APISuite/numbers_rdd_tests_v2api)
          2017-03-23 18:42:47,006 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] DEBUG s3a.S3AFileSystem (S3AFileSystem.java:s3GetFileStatus(1905)) - Not Found: s3a://hwdev-steve-new/spark-cloud/S3ANumbersSuiteV2APISuite/numbers_rdd_tests_v2api
          2017-03-23 18:42:47,006 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] INFO  commit.DurationInfo (DurationInfo.java:close(44)) - Task committer attempt_20170323184243_0000_m_000000_0: preparing to commit Job: 0:00:617
          2017-03-23 18:42:47,007 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] INFO  commit.DurationInfo (DurationInfo.java:<init>(33)) - Starting Task committer attempt_20170323184243_0000_m_000000_0: committing Job job_20170323184242_0001
          2017-03-23 18:42:47,007 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] DEBUG staging.StagingS3GuardCommitter (StagingS3GuardCommitter.java:commitJobInternal(649)) - Task committer attempt_20170323184243_0000_m_000000_0: committing the output of 1 task(s)
          2017-03-23 18:42:47,010 [s3-committer-pool-1] DEBUG staging.S3Util (S3Util.java:finishCommit(101)) - Finish PendingUpload{ to 's3a://hwdev-steve-new/spark-cloud/S3ANumbersSuiteV2APISuite/numbers_rdd_tests_v2api/part-r-00000-job_20170323184243_0000' , uploadId='_P7n0N2FxTdVYiTBBFNB2Y9TOQgnbuKT1Lt.mwfx4pdmT6aK4E5XlxfpFZNwSUv0viLzNx45qvhJAqDyhSDqajozcfuFesRF2enq.bpkfc0-'}
          2017-03-23 18:42:47,364 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] DEBUG s3a.S3AFileSystem (S3AFileSystem.java:innerGetFileStatus(1782)) - Getting path status for s3a://hwdev-steve-new/spark-cloud/S3ANumbersSuiteV2APISuite/numbers_rdd_tests_v2api/_temporary  (spark-cloud/S3ANumbersSuiteV2APISuite/numbers_rdd_tests_v2api/_temporary)
          2017-03-23 18:42:47,941 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] DEBUG s3a.S3AFileSystem (S3AFileSystem.java:s3GetFileStatus(1905)) - Not Found: s3a://hwdev-steve-new/spark-cloud/S3ANumbersSuiteV2APISuite/numbers_rdd_tests_v2api/_temporary
          2017-03-23 18:42:47,942 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] DEBUG s3a.S3AFileSystem (S3AFileSystem.java:delete(1418)) - Couldn't delete s3a://hwdev-steve-new/spark-cloud/S3ANumbersSuiteV2APISuite/numbers_rdd_tests_v2api/_temporary - does not exist
          2017-03-23 18:42:47,943 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] INFO  commit.DurationInfo (DurationInfo.java:close(44)) - Task committer attempt_20170323184243_0000_m_000000_0: committing Job job_20170323184242_0001: 0:00:936
          2017-03-23 18:42:47,943 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] DEBUG commit.FileCommitActions (FileCommitActions.java:touchSuccessMarker(231)) - Touching success marker for job s3a://hwdev-steve-new/spark-cloud/S3ANumbersSuiteV2APISuite/numbers_rdd_tests_v2api/_SUCCESS
          2017-03-23 18:42:47,944 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] DEBUG s3a.S3AFileSystem (S3AFileSystem.java:innerGetFileStatus(1782)) - Getting path status for s3a://hwdev-steve-new/spark-cloud/S3ANumbersSuiteV2APISuite/numbers_rdd_tests_v2api/_SUCCESS  (spark-cloud/S3ANumbersSuiteV2APISuite/numbers_rdd_tests_v2api/_SUCCESS)
          2017-03-23 18:42:48,494 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] DEBUG s3a.S3AFileSystem (S3AFileSystem.java:s3GetFileStatus(1905)) - Not Found: s3a://hwdev-steve-new/spark-cloud/S3ANumbersSuiteV2APISuite/numbers_rdd_tests_v2api/_SUCCESS
          2017-03-23 18:42:48,495 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] DEBUG commit.DelayedCommitFSIntegration (DelayedCommitFSIntegration.java:getTracker(98)) - Created DefaultPutTracker{destKey='spark-cloud/S3ANumbersSuiteV2APISuite/numbers_rdd_tests_v2api/_SUCCESS'}
          2017-03-23 18:42:48,498 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] DEBUG s3a.S3AFileSystem (S3AFileSystem.java:createWriteOperationHelper(703)) - Created WriteOperationHelper {bucket=hwdev-steve-new, key='spark-cloud/S3ANumbersSuiteV2APISuite/numbers_rdd_tests_v2api/_SUCCESS'}
          2017-03-23 18:42:48,514 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] DEBUG s3a.S3ADataBlocks (S3ADataBlocks.java:startUpload(324)) - Start datablock[1] upload
          2017-03-23 18:42:48,514 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] DEBUG s3a.S3ADataBlocks (S3ADataBlocks.java:enterState(231)) - FileBlock{index=1, destFile=/Users/stevel/Projects/sparkwork/spark-cloud-examples/cloud-examples/target/tmp/s3ablock-0001-4746647579488981641.tmp, state=Writing, dataSize=0, limit=8388608}: entering state Upload
          2017-03-23 18:42:48,518 [s3a-transfer-shared-pool1-t1] DEBUG s3a.S3AFileSystem (S3AFileSystem.java:putObjectDirect(1280)) - PUT 0 bytes to spark-cloud/S3ANumbersSuiteV2APISuite/numbers_rdd_tests_v2api/_SUCCESS
          2017-03-23 18:42:48,519 [s3a-transfer-shared-pool1-t1] DEBUG s3a.S3AFileSystem (S3AFileSystem.java:incrementPutStartStatistics(1321)) - PUT start 0 bytes
          2017-03-23 18:42:48,740 [s3a-transfer-shared-pool1-t1] DEBUG s3a.S3AFileSystem (S3AFileSystem.java:incrementPutCompletedStatistics(1338)) - PUT completed success=true; 0 bytes
          2017-03-23 18:42:48,740 [s3a-transfer-shared-pool1-t1] DEBUG s3a.S3AFileSystem (S3AFileSystem.java:finishedWrite(2183)) - Finished write to spark-cloud/S3ANumbersSuiteV2APISuite/numbers_rdd_tests_v2api/_SUCCESS, len 0
          2017-03-23 18:42:48,938 [s3a-transfer-shared-pool1-t1] DEBUG s3a.S3ADataBlocks (S3ADataBlocks.java:enterState(231)) - FileBlock{index=1, destFile=/Users/stevel/Projects/sparkwork/spark-cloud-examples/cloud-examples/target/tmp/s3ablock-0001-4746647579488981641.tmp, state=Upload, dataSize=0, limit=8388608}: entering state Closed
          2017-03-23 18:42:48,938 [s3a-transfer-shared-pool1-t1] DEBUG s3a.S3ADataBlocks (S3ADataBlocks.java:close(346)) - Closed FileBlock{index=1, destFile=/Users/stevel/Projects/sparkwork/spark-cloud-examples/cloud-examples/target/tmp/s3ablock-0001-4746647579488981641.tmp, state=Closed, dataSize=0, limit=8388608}
          2017-03-23 18:42:48,938 [s3a-transfer-shared-pool1-t1] DEBUG s3a.S3ADataBlocks (S3ADataBlocks.java:innerClose(886)) - Closing FileBlock{index=1, destFile=/Users/stevel/Projects/sparkwork/spark-cloud-examples/cloud-examples/target/tmp/s3ablock-0001-4746647579488981641.tmp, state=Closed, dataSize=0, limit=8388608}
          2017-03-23 18:42:48,939 [s3a-transfer-shared-pool1-t1] DEBUG s3a.S3ADataBlocks (S3ADataBlocks.java:closeBlock(940)) - block[1]: closeBlock()
          2017-03-23 18:42:48,939 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] DEBUG s3a.S3AFileSystem (S3AFileSystem.java:writeSuccessful(2701)) - Successful write to spark-cloud/S3ANumbersSuiteV2APISuite/numbers_rdd_tests_v2api/_SUCCESS, len 0
          2017-03-23 18:42:48,941 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] DEBUG s3a.S3AFileSystem (S3AFileSystem.java:innerGetFileStatus(1782)) - Getting path status for s3a://hwdev-steve-new/spark-cloud/S3ANumbersSuiteV2APISuite/numbers_rdd_tests_v2api/_temporary-1  (spark-cloud/S3ANumbersSuiteV2APISuite/numbers_rdd_tests_v2api/_temporary-1)
          2017-03-23 18:42:50,072 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] DEBUG s3a.S3AFileSystem (S3AFileSystem.java:s3GetFileStatus(1905)) - Not Found: s3a://hwdev-steve-new/spark-cloud/S3ANumbersSuiteV2APISuite/numbers_rdd_tests_v2api/_temporary-1
          2017-03-23 18:42:50,072 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] DEBUG s3a.S3AFileSystem (S3AFileSystem.java:delete(1418)) - Couldn't delete s3a://hwdev-steve-new/spark-cloud/S3ANumbersSuiteV2APISuite/numbers_rdd_tests_v2api/_temporary-1 - does not exist
          2017-03-23 18:42:50,074 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] INFO  io.SparkHadoopMapReduceWriter (Logging.scala:logInfo(54)) - Job job_20170323184242_0001 committed.
          2017-03-23 18:42:50,081 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] INFO  ui.SparkUI (Logging.scala:logInfo(54)) - Stopped Spark web UI at http://192.168.1.38:4040
          2017-03-23 18:42:50,108 [dispatcher-event-loop-2] INFO  spark.MapOutputTrackerMasterEndpoint (Logging.scala:logInfo(54)) - MapOutputTrackerMasterEndpoint stopped!
          2017-03-23 18:42:50,117 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] INFO  memory.MemoryStore (Logging.scala:logInfo(54)) - MemoryStore cleared
          2017-03-23 18:42:50,118 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] INFO  storage.BlockManager (Logging.scala:logInfo(54)) - BlockManager stopped
          2017-03-23 18:42:50,123 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] INFO  storage.BlockManagerMaster (Logging.scala:logInfo(54)) - BlockManagerMaster stopped
          2017-03-23 18:42:50,125 [dispatcher-event-loop-0] INFO  scheduler.OutputCommitCoordinator$OutputCommitCoordinatorEndpoint (Logging.scala:logInfo(54)) - OutputCommitCoordinator stopped!
          2017-03-23 18:42:50,128 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] INFO  spark.SparkContext (Logging.scala:logInfo(54)) - Successfully stopped SparkContext
          2017-03-23 18:42:50,130 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] INFO  s3.S3ANumbersSuiteV2APISuite (CloudLogging.scala:logInfo(56)) - Filesystem statistics
           S3AFileSystem
          {uri=s3a://hwdev-steve-new, workingDir=s3a://hwdev-steve-new/user/stevel, inputPolicy=sequential, partSize=8388608, enableMultiObjectsDelete=true, maxKeys=5000, readAhead=65536, blockSize=1048576, multiPartThreshold=2147483647, serverSideEncryptionAlgorithm='NONE', blockFactory=org.apache.hadoop.fs.s3a.S3ADataBlocks$DiskBlockFactory@6707a4bf, metastore=NullMetadataStore, authoritative=false, boundedExecutor=BlockingThreadPoolExecutorService
          {SemaphoredDelegatingExecutor
          {permitCount=25, available=25, waiting=0}, activeCount=0}, unboundedExecutor=java.util.concurrent.ThreadPoolExecutor@3fc7c734[Running, pool size = 0, active threads = 0, queued tasks = 0, completed tasks = 0], statistics 
          {0 bytes read, 0 bytes written, 7 read ops, 0 large read ops, 3 write ops}, metrics 
          {
          {Context=S3AFileSystem} 
          {FileSystemId=58e091c4-0ae9-4626-8ecf-0d1ac3ffdb27-hwdev-steve-new} 
          {fsURI=s3a://hwdev-steve-new} 
          {files_created=1} 
          {files_copied=0} 
          {files_copied_bytes=0} 
          {files_deleted=0} 
          {fake_directories_deleted=3} 
          {directories_created=0} 
          {directories_deleted=0} 
          {ignored_errors=3} 
          {op_copy_from_local_file=0} 
          {op_exists=3} 
          {op_get_file_status=7} 
          {op_glob_status=0} 
          {op_is_directory=0} 
          {op_is_file=0} 
          {op_list_files=0} 
          {op_list_located_status=0} 
          {op_list_status=0} 
          {op_mkdirs=0} 
          {op_rename=0} 
          {object_copy_requests=0} 
          {object_delete_requests=1} 
          {object_list_requests=7} 
          {object_continue_list_requests=0} 
          {object_metadata_requests=14} 
          {object_multipart_aborted=0} 
          {object_put_bytes=0} 
          {object_put_requests=1} 
          {object_put_requests_completed=1} 
          {stream_write_failures=0} 
          {stream_write_block_uploads=0} 
          {stream_write_block_uploads_committed=0} 
          {stream_write_block_uploads_aborted=0} 
          {stream_write_total_time=0} 
          {stream_write_total_data=0} 
          {object_put_requests_active=0} 
          {object_put_bytes_pending=0} 
          {stream_write_block_uploads_active=0} 
          {stream_write_block_uploads_pending=1} 
          {stream_write_block_uploads_data_pending=0} 
          {stream_read_fully_operations=0} 
          {stream_opened=0} 
          {stream_bytes_skipped_on_seek=0} 
          {stream_closed=0} 
          {stream_bytes_backwards_on_seek=0} 
          {stream_bytes_read=0} 
          {stream_read_operations_incomplete=0} 
          {stream_bytes_discarded_in_abort=0} 
          {stream_close_operations=0} 
          {stream_read_operations=0} 
          {stream_aborted=0} 
          {stream_forward_seek_operations=0} 
          {stream_backward_seek_operations=0} 
          {stream_seek_operations=0} 
          {stream_bytes_read_in_close=0} 
          {stream_read_exceptions=0} }}
          2017-03-23 18:42:50,130 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] DEBUG s3a.S3AFileSystem (S3AFileSystem.java:listFiles(2505)) - listFiles(s3a://hwdev-steve-new/spark-cloud/S3ANumbersSuiteV2APISuite/numbers_rdd_tests_v2api, true)
          2017-03-23 18:42:50,130 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] DEBUG s3a.S3AFileSystem (S3AFileSystem.java:innerGetFileStatus(1782)) - Getting path status for s3a://hwdev-steve-new/spark-cloud/S3ANumbersSuiteV2APISuite/numbers_rdd_tests_v2api  (spark-cloud/S3ANumbersSuiteV2APISuite/numbers_rdd_tests_v2api)
          2017-03-23 18:42:50,761 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] DEBUG s3a.S3AFileSystem (S3AFileSystem.java:s3GetFileStatus(1880)) - Found path as directory (with /): 0/1
          2017-03-23 18:42:50,761 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] DEBUG s3a.S3AFileSystem (S3AFileSystem.java:s3GetFileStatus(1885)) - Summary: spark-cloud/S3ANumbersSuiteV2APISuite/numbers_rdd_tests_v2api/_SUCCESS 0
          2017-03-23 18:42:50,763 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] DEBUG s3a.S3AFileSystem (S3AFileSystem.java:listFiles(2518)) - Requesting all entries under spark-cloud/S3ANumbersSuiteV2APISuite/numbers_rdd_tests_v2api/ with delimiter 'null'
          2017-03-23 18:42:50,953 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] DEBUG s3a.S3AFileSystem (Listing.java:buildNextStatusBatch(296)) - s3a://hwdev-steve-new/spark-cloud/S3ANumbersSuiteV2APISuite/numbers_rdd_tests_v2api/_SUCCESS: spark-cloud/S3ANumbersSuiteV2APISuite/numbers_rdd_tests_v2api/_SUCCESS size=0
          2017-03-23 18:42:50,954 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] DEBUG s3a.S3AFileSystem (Listing.java:buildNextStatusBatch(302)) - Adding: S3AFileStatus{path=s3a://hwdev-steve-new/spark-cloud/S3ANumbersSuiteV2APISuite/numbers_rdd_tests_v2api/_SUCCESS; isDirectory=false; length=0; replication=1; blocksize=1048576; modification_time=1490294569000; access_time=0; owner=stevel; group=stevel; permission=rw-rw-rw-; isSymlink=false} isEmptyDirectory=FALSE
          2017-03-23 18:42:50,954 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] DEBUG s3a.S3AFileSystem (Listing.java:buildNextStatusBatch(296)) - s3a://hwdev-steve-new/spark-cloud/S3ANumbersSuiteV2APISuite/numbers_rdd_tests_v2api/part-r-00000-job_20170323184243_0000: spark-cloud/S3ANumbersSuiteV2APISuite/numbers_rdd_tests_v2api/part-r-00000-job_20170323184243_0000 size=515303
          2017-03-23 18:42:50,955 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] DEBUG s3a.S3AFileSystem (Listing.java:buildNextStatusBatch(302)) - Adding: S3AFileStatus{path=s3a://hwdev-steve-new/spark-cloud/S3ANumbersSuiteV2APISuite/numbers_rdd_tests_v2api/part-r-00000-job_20170323184243_0000; isDirectory=false; length=515303; replication=1; blocksize=1048576; modification_time=1490294565000; access_time=0; owner=stevel; group=stevel; permission=rw-rw-rw-; isSymlink=false} isEmptyDirectory=FALSE
          2017-03-23 18:42:50,955 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] DEBUG s3a.S3AFileSystem (Listing.java:buildNextStatusBatch(330)) - Added 2 entries; ignored 0; hasNext=true; hasMoreObjects=false
          2017-03-23 18:42:50,957 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] INFO  s3.S3ANumbersSuiteV2APISuite (CloudLogging.scala:logInfo(56)) - Contents of s3a://hwdev-steve-new/spark-cloud/S3ANumbersSuiteV2APISuite/numbers_rdd_tests_v2api:
            s3a://hwdev-steve-new/spark-cloud/S3ANumbersSuiteV2APISuite/numbers_rdd_tests_v2api/_SUCCESS [0]
            s3a://hwdev-steve-new/spark-cloud/S3ANumbersSuiteV2APISuite/numbers_rdd_tests_v2api/part-r-00000-job_20170323184243_0000 [515303]
          Dirs: 0, Files: 2, total bytes = 515303
          2017-03-23 18:42:50,964 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] INFO  spark.SparkContext (Logging.scala:logInfo(54)) - SparkContext already stopped.
          2017-03-23 18:42:50,966 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] DEBUG s3a.S3AFileSystem (S3AFileSystem.java:innerGetFileStatus(1782)) - Getting path status for s3a://hwdev-steve-new/spark-cloud/S3ANumbersSuiteV2APISuite  (spark-cloud/S3ANumbersSuiteV2APISuite)
          - SaveRDD
          + Cleaning s3a://hwdev-steve-new/spark-cloud/S3ANumbersSuiteV2APISuite 
          2017-03-23 18:42:51,514 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] DEBUG s3a.S3AFileSystem (S3AFileSystem.java:s3GetFileStatus(1880)) - Found path as directory (with /): 1/0
          2017-03-23 18:42:51,514 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] DEBUG s3a.S3AFileSystem (S3AFileSystem.java:s3GetFileStatus(1888)) - Prefix: spark-cloud/S3ANumbersSuiteV2APISuite/numbers_rdd_tests_v2api/
          2017-03-23 18:42:51,514 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] DEBUG s3a.S3AFileSystem (S3AFileSystem.java:innerGetFileStatus(1782)) - Getting path status for s3a://hwdev-steve-new/spark-cloud/S3ANumbersSuiteV2APISuite  (spark-cloud/S3ANumbersSuiteV2APISuite)
          2017-03-23 18:42:52,059 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] DEBUG s3a.S3AFileSystem (S3AFileSystem.java:s3GetFileStatus(1880)) - Found path as directory (with /): 1/0
          2017-03-23 18:42:52,059 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] DEBUG s3a.S3AFileSystem (S3AFileSystem.java:s3GetFileStatus(1888)) - Prefix: spark-cloud/S3ANumbersSuiteV2APISuite/numbers_rdd_tests_v2api/
          2017-03-23 18:42:52,060 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] DEBUG s3a.S3AFileSystem (S3AFileSystem.java:innerDelete(1444)) - Delete path s3a://hwdev-steve-new/spark-cloud/S3ANumbersSuiteV2APISuite - recursive true
          2017-03-23 18:42:52,060 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] DEBUG s3a.S3AFileSystem (S3AFileSystem.java:innerDelete(1449)) - delete: Path is a directory: s3a://hwdev-steve-new/spark-cloud/S3ANumbersSuiteV2APISuite
          2017-03-23 18:42:52,060 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] DEBUG s3a.S3AFileSystem (S3AFileSystem.java:innerDelete(1473)) - Getting objects for directory prefix spark-cloud/S3ANumbersSuiteV2APISuite/ to delete
          2017-03-23 18:42:52,248 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] DEBUG s3a.S3AFileSystem (S3AFileSystem.java:innerDelete(1483)) - Got object to delete spark-cloud/S3ANumbersSuiteV2APISuite/numbers_rdd_tests_v2api/_SUCCESS
          2017-03-23 18:42:52,248 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] DEBUG s3a.S3AFileSystem (S3AFileSystem.java:innerDelete(1483)) - Got object to delete spark-cloud/S3ANumbersSuiteV2APISuite/numbers_rdd_tests_v2api/part-r-00000-job_20170323184243_0000
          2017-03-23 18:42:53,499 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] DEBUG s3a.S3AFileSystem (S3AFileSystem.java:s3GetFileStatus(1880)) - Found path as directory (with /): 1/0
          2017-03-23 18:42:53,500 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] DEBUG s3a.S3AFileSystem (S3AFileSystem.java:s3GetFileStatus(1888)) - Prefix: spark-cloud/S3ALineCountWritebackSuite/
          Run completed in 15 seconds, 386 milliseconds.
          
          Show
          stevel@apache.org Steve Loughran added a comment - And for the very curious, here's a trace of a test run of the latest code doing some simple sparkContact.makeRDD(1 to count).saveAsTextFile("s3a:///something") with the directory committer. There's a job committer and task committer in the logs, hence the role log info to work out whats happening. fs.s3a mostly at debug, except for instrumentation. FS stats exclude all the info on the uploads because they (currently) go straight through the s3a client. ------------------------------------------- 2017-03-23 18:42:41,455 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] DEBUG s3a.S3AFileSystem (S3AFileSystem.java:innerGetFileStatus(1782)) - Getting path status for s3a: //hwdev-steve- new /spark-cloud/S3ANumbersSuiteV2APISuite/numbers_rdd_tests_v2api (spark-cloud/S3ANumbersSuiteV2APISuite/numbers_rdd_tests_v2api) 2017-03-23 18:42:42,024 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] DEBUG s3a.S3AFileSystem (S3AFileSystem.java:s3GetFileStatus(1905)) - Not Found: s3a: //hwdev-steve- new /spark-cloud/S3ANumbersSuiteV2APISuite/numbers_rdd_tests_v2api 2017-03-23 18:42:42,024 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] DEBUG s3a.S3AFileSystem (S3AFileSystem.java:delete(1418)) - Couldn't delete s3a: //hwdev-steve- new /spark-cloud/S3ANumbersSuiteV2APISuite/numbers_rdd_tests_v2api - does not exist 2017-03-23 18:42:42,024 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] INFO s3.S3ANumbersSuiteV2APISuite (CloudLogging.scala:logInfo(56)) - Switching to local file: // fs for default FS 2017-03-23 18:42:42,098 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] INFO spark.SparkContext (Logging.scala:logInfo(54)) - Running Spark version 2.2.0-SNAPSHOT 2017-03-23 18:42:42,133 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] INFO spark.SparkContext (Logging.scala:logInfo(54)) - Submitted application: test 2017-03-23 18:42:42,147 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] INFO spark. SecurityManager (Logging.scala:logInfo(54)) - Changing view acls to: stevel 2017-03-23 18:42:42,147 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] INFO spark. SecurityManager (Logging.scala:logInfo(54)) - Changing modify acls to: stevel 2017-03-23 18:42:42,148 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] INFO spark. SecurityManager (Logging.scala:logInfo(54)) - Changing view acls groups to: 2017-03-23 18:42:42,149 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] INFO spark. SecurityManager (Logging.scala:logInfo(54)) - Changing modify acls groups to: 2017-03-23 18:42:42,150 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] INFO spark. SecurityManager (Logging.scala:logInfo(54)) - SecurityManager : authentication disabled; ui acls disabled; users with view permissions: Set(stevel); groups with view permissions: Set(); users with modify permissions: Set(stevel); groups with modify permissions: Set() 2017-03-23 18:42:42,369 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] INFO util.Utils (Logging.scala:logInfo(54)) - Successfully started service 'sparkDriver' on port 54392. 2017-03-23 18:42:42,389 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] INFO spark.SparkEnv (Logging.scala:logInfo(54)) - Registering MapOutputTracker 2017-03-23 18:42:42,402 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] INFO spark.SparkEnv (Logging.scala:logInfo(54)) - Registering BlockManagerMaster 2017-03-23 18:42:42,405 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] INFO storage.BlockManagerMasterEndpoint (Logging.scala:logInfo(54)) - Using org.apache.spark.storage.DefaultTopologyMapper for getting topology information 2017-03-23 18:42:42,405 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] INFO storage.BlockManagerMasterEndpoint (Logging.scala:logInfo(54)) - BlockManagerMasterEndpoint up 2017-03-23 18:42:42,412 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] INFO storage.DiskBlockManager (Logging.scala:logInfo(54)) - Created local directory at /Users/stevel/Projects/sparkwork/spark-cloud-examples/cloud-examples/target/tmp/blockmgr-9ab575c8-964f-4cb7-96cb-e5c6efc04d60 2017-03-23 18:42:42,429 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] INFO memory.MemoryStore (Logging.scala:logInfo(54)) - MemoryStore started with capacity 2004.6 MB 2017-03-23 18:42:42,466 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] INFO spark.SparkEnv (Logging.scala:logInfo(54)) - Registering OutputCommitCoordinator 2017-03-23 18:42:42,579 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] INFO util.Utils (Logging.scala:logInfo(54)) - Successfully started service 'SparkUI' on port 4040. 2017-03-23 18:42:42,606 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] INFO ui.SparkUI (Logging.scala:logInfo(54)) - Bound SparkUI to 0.0.0.0, and started at http: //192.168.1.38:4040 2017-03-23 18:42:42,679 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] INFO executor.Executor (Logging.scala:logInfo(54)) - Starting executor ID driver on host localhost 2017-03-23 18:42:42,694 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] INFO util.Utils (Logging.scala:logInfo(54)) - Successfully started service 'org.apache.spark.network.netty.NettyBlockTransferService' on port 54394. 2017-03-23 18:42:42,695 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] INFO netty.NettyBlockTransferService (Logging.scala:logInfo(54)) - Server created on 192.168.1.38:54394 2017-03-23 18:42:42,696 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] INFO storage.BlockManager (Logging.scala:logInfo(54)) - Using org.apache.spark.storage.RandomBlockReplicationPolicy for block replication policy 2017-03-23 18:42:42,697 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] INFO storage.BlockManagerMaster (Logging.scala:logInfo(54)) - Registering BlockManager BlockManagerId(driver, 192.168.1.38, 54394, None) 2017-03-23 18:42:42,701 [dispatcher-event-loop-2] INFO storage.BlockManagerMasterEndpoint (Logging.scala:logInfo(54)) - Registering block manager 192.168.1.38:54394 with 2004.6 MB RAM, BlockManagerId(driver, 192.168.1.38, 54394, None) 2017-03-23 18:42:42,703 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] INFO storage.BlockManagerMaster (Logging.scala:logInfo(54)) - Registered BlockManager BlockManagerId(driver, 192.168.1.38, 54394, None) 2017-03-23 18:42:42,703 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] INFO storage.BlockManager (Logging.scala:logInfo(54)) - Initialized BlockManager: BlockManagerId(driver, 192.168.1.38, 54394, None) 2017-03-23 18:42:42,898 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] INFO s3.S3ANumbersSuiteV2APISuite (CloudLogging.scala:logInfo(56)) - Generating output under s3a: //hwdev-steve- new /spark-cloud/S3ANumbersSuiteV2APISuite/numbers_rdd_tests_v2api 2017-03-23 18:42:42,940 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] DEBUG s3a.S3AFileSystem (S3AFileSystem.java:innerGetFileStatus(1782)) - Getting path status for s3a: //hwdev-steve- new /spark-cloud/S3ANumbersSuiteV2APISuite/numbers_rdd_tests_v2api (spark-cloud/S3ANumbersSuiteV2APISuite/numbers_rdd_tests_v2api) 2017-03-23 18:42:43,486 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] DEBUG s3a.S3AFileSystem (S3AFileSystem.java:s3GetFileStatus(1905)) - Not Found: s3a: //hwdev-steve- new /spark-cloud/S3ANumbersSuiteV2APISuite/numbers_rdd_tests_v2api 2017-03-23 18:42:43,492 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] DEBUG output.PathOutputCommitterFactory (PathOutputCommitterFactory.java:getOutputCommitterFactory(92)) - Using FileOutputCommitter factory class class org.apache.hadoop.fs.s3a.commit.staging.DirectoryStagingCommitterFactory 2017-03-23 18:42:43,495 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] DEBUG commit.AbstractS3GuardCommitter (AbstractS3GuardCommitter.java:<init>(85)) - Task committer attempt_20170323184243_0000_m_000000_0 instantiated for job "" ID job_20170323184243_0000 with destination s3a: //hwdev-steve- new /spark-cloud/S3ANumbersSuiteV2APISuite/numbers_rdd_tests_v2api 2017-03-23 18:42:43,497 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] DEBUG commit.AbstractS3GuardCommitter (AbstractS3GuardCommitter.java:<init>(115)) - Task committer attempt_20170323184243_0000_m_000000_0} instantiated for "" ID job_20170323184243_0000 2017-03-23 18:42:43,499 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] DEBUG commit.AbstractS3GuardCommitter (AbstractS3GuardCommitter.java:setWorkPath(162)) - Setting work path to file:/Users/stevel/Projects/sparkwork/spark-cloud-examples/cloud-examples/target/test/mapred/local/job_20170323184243_0000/0/attempt_20170323184243_0000_m_000000_0/_temporary/0/_temporary/attempt_20170323184243_0000_m_000000_0 2017-03-23 18:42:43,501 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] INFO output.FileOutputCommitter (FileOutputCommitter.java:<init>(123)) - File Output Committer Algorithm version is 1 2017-03-23 18:42:43,501 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] INFO output.FileOutputCommitter (FileOutputCommitter.java:<init>(138)) - FileOutputCommitter skip cleanup _temporary folders under output directory: false , ignore cleanup failures: true 2017-03-23 18:42:43,502 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] DEBUG staging.StagingS3GuardCommitter (StagingS3GuardCommitter.java:postCreationActions(168)) - Conflict resolution mode: FAIL 2017-03-23 18:42:43,502 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] INFO commit.Abstract3GuardCommitterFactory (Abstract3GuardCommitterFactory.java:createOutputCommitter(61)) - Using Commmitter StagingS3GuardCommitter{AbstractS3GuardCommitter{role=Task committer attempt_20170323184243_0000_m_000000_0outputPath=s3a: //hwdev-steve- new /spark-cloud/S3ANumbersSuiteV2APISuite/numbers_rdd_tests_v2api, workPath=file:/Users/stevel/Projects/sparkwork/spark-cloud-examples/cloud-examples/target/test/mapred/local/job_20170323184243_0000/0/attempt_20170323184243_0000_m_000000_0/_temporary/0/_temporary/attempt_20170323184243_0000_m_000000_0}, finalOutputPath= null , conflictResolution=FAIL, wrappedCommitter=FileOutputCommitter{outputPath=file:/Users/stevel/Projects/sparkwork/spark-cloud-examples/cloud-examples/target/tmp/stevel/job_20170323184243_0000/staging-uploads, workPath= null , algorithmVersion=1, skipCleanup= false , ignoreCleanupFailures= true }} for s3a://hwdev-steve- new /spark-cloud/S3ANumbersSuiteV2APISuite/numbers_rdd_tests_v2api 2017-03-23 18:42:43,502 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] DEBUG staging.StagingS3GuardCommitter (StagingS3GuardCommitter.java:setupJob(520)) - Task committer attempt_20170323184243_0000_m_000000_0, Setting up job job_20170323184242_0001 2017-03-23 18:42:43,539 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] DEBUG staging.StagingS3GuardCommitter (StagingS3GuardCommitter.java:getOutputPath(1057)) - Task committer attempt_20170323184243_0000_m_000000_0: final output path is s3a: //hwdev-steve- new /spark-cloud/S3ANumbersSuiteV2APISuite/numbers_rdd_tests_v2api 2017-03-23 18:42:43,539 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] DEBUG s3a.S3AFileSystem (S3AFileSystem.java:innerGetFileStatus(1782)) - Getting path status for s3a: //hwdev-steve- new /spark-cloud/S3ANumbersSuiteV2APISuite/numbers_rdd_tests_v2api (spark-cloud/S3ANumbersSuiteV2APISuite/numbers_rdd_tests_v2api) 2017-03-23 18:42:44,084 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] DEBUG s3a.S3AFileSystem (S3AFileSystem.java:s3GetFileStatus(1905)) - Not Found: s3a: //hwdev-steve- new /spark-cloud/S3ANumbersSuiteV2APISuite/numbers_rdd_tests_v2api 2017-03-23 18:42:44,122 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] INFO spark.SparkContext (Logging.scala:logInfo(54)) - Starting job: runJob at SparkHadoopMapReduceWriter.scala:88 2017-03-23 18:42:44,131 [dag-scheduler-event-loop] INFO scheduler.DAGScheduler (Logging.scala:logInfo(54)) - Got job 0 (runJob at SparkHadoopMapReduceWriter.scala:88) with 1 output partitions 2017-03-23 18:42:44,132 [dag-scheduler-event-loop] INFO scheduler.DAGScheduler (Logging.scala:logInfo(54)) - Final stage: ResultStage 0 (runJob at SparkHadoopMapReduceWriter.scala:88) 2017-03-23 18:42:44,132 [dag-scheduler-event-loop] INFO scheduler.DAGScheduler (Logging.scala:logInfo(54)) - Parents of final stage: List() 2017-03-23 18:42:44,133 [dag-scheduler-event-loop] INFO scheduler.DAGScheduler (Logging.scala:logInfo(54)) - Missing parents: List() 2017-03-23 18:42:44,137 [dag-scheduler-event-loop] INFO scheduler.DAGScheduler (Logging.scala:logInfo(54)) - Submitting ResultStage 0 (MapPartitionsRDD[1] at map at NumbersRddTests.scala:120), which has no missing parents 2017-03-23 18:42:44,208 [dag-scheduler-event-loop] INFO memory.MemoryStore (Logging.scala:logInfo(54)) - Block broadcast_0 stored as values in memory (estimated size 74.9 KB, free 2004.5 MB) 2017-03-23 18:42:44,232 [dag-scheduler-event-loop] INFO memory.MemoryStore (Logging.scala:logInfo(54)) - Block broadcast_0_piece0 stored as bytes in memory (estimated size 25.0 KB, free 2004.5 MB) 2017-03-23 18:42:44,234 [dispatcher-event-loop-4] INFO storage.BlockManagerInfo (Logging.scala:logInfo(54)) - Added broadcast_0_piece0 in memory on 192.168.1.38:54394 (size: 25.0 KB, free: 2004.6 MB) 2017-03-23 18:42:44,236 [dag-scheduler-event-loop] INFO spark.SparkContext (Logging.scala:logInfo(54)) - Created broadcast 0 from broadcast at DAGScheduler.scala:1002 2017-03-23 18:42:44,251 [dag-scheduler-event-loop] INFO scheduler.DAGScheduler (Logging.scala:logInfo(54)) - Submitting 1 missing tasks from ResultStage 0 (MapPartitionsRDD[1] at map at NumbersRddTests.scala:120) (first 15 tasks are for partitions Vector(0)) 2017-03-23 18:42:44,252 [dag-scheduler-event-loop] INFO scheduler.TaskSchedulerImpl (Logging.scala:logInfo(54)) - Adding task set 0.0 with 1 tasks 2017-03-23 18:42:44,281 [dispatcher-event-loop-5] INFO scheduler.TaskSetManager (Logging.scala:logInfo(54)) - Starting task 0.0 in stage 0.0 (TID 0, localhost, executor driver, partition 0, PROCESS_LOCAL, 7598 bytes) 2017-03-23 18:42:44,286 [Executor task launch worker for task 0] INFO executor.Executor (Logging.scala:logInfo(54)) - Running task 0.0 in stage 0.0 (TID 0) 2017-03-23 18:42:44,333 [Executor task launch worker for task 0] DEBUG output.PathOutputCommitterFactory (PathOutputCommitterFactory.java:getOutputCommitterFactory(92)) - Using FileOutputCommitter factory class class org.apache.hadoop.fs.s3a.commit.staging.DirectoryStagingCommitterFactory 2017-03-23 18:42:44,334 [Executor task launch worker for task 0] DEBUG commit.AbstractS3GuardCommitter (AbstractS3GuardCommitter.java:<init>(85)) - Task committer attempt_20170323184242_0000_r_000000_0 instantiated for job "" ID job_20170323184242_0000 with destination s3a: //hwdev-steve- new /spark-cloud/S3ANumbersSuiteV2APISuite/numbers_rdd_tests_v2api 2017-03-23 18:42:44,335 [Executor task launch worker for task 0] DEBUG commit.AbstractS3GuardCommitter (AbstractS3GuardCommitter.java:<init>(115)) - Task committer attempt_20170323184242_0000_r_000000_0} instantiated for "" ID job_20170323184242_0000 2017-03-23 18:42:44,336 [Executor task launch worker for task 0] DEBUG commit.AbstractS3GuardCommitter (AbstractS3GuardCommitter.java:setWorkPath(162)) - Setting work path to file:/Users/stevel/Projects/sparkwork/spark-cloud-examples/cloud-examples/target/test/mapred/local/job_20170323184243_0000/0/attempt_20170323184242_0000_r_000000_0/_temporary/0/_temporary/attempt_20170323184242_0000_r_000000_0 2017-03-23 18:42:44,337 [Executor task launch worker for task 0] INFO output.FileOutputCommitter (FileOutputCommitter.java:<init>(123)) - File Output Committer Algorithm version is 1 2017-03-23 18:42:44,337 [Executor task launch worker for task 0] INFO output.FileOutputCommitter (FileOutputCommitter.java:<init>(138)) - FileOutputCommitter skip cleanup _temporary folders under output directory: false , ignore cleanup failures: true 2017-03-23 18:42:44,337 [Executor task launch worker for task 0] DEBUG staging.StagingS3GuardCommitter (StagingS3GuardCommitter.java:postCreationActions(168)) - Conflict resolution mode: FAIL 2017-03-23 18:42:44,338 [Executor task launch worker for task 0] INFO commit.Abstract3GuardCommitterFactory (Abstract3GuardCommitterFactory.java:createOutputCommitter(61)) - Using Commmitter StagingS3GuardCommitter{AbstractS3GuardCommitter{role=Task committer attempt_20170323184242_0000_r_000000_0outputPath=s3a: //hwdev-steve- new /spark-cloud/S3ANumbersSuiteV2APISuite/numbers_rdd_tests_v2api, workPath=file:/Users/stevel/Projects/sparkwork/spark-cloud-examples/cloud-examples/target/test/mapred/local/job_20170323184243_0000/0/attempt_20170323184242_0000_r_000000_0/_temporary/0/_temporary/attempt_20170323184242_0000_r_000000_0}, finalOutputPath= null , conflictResolution=FAIL, wrappedCommitter=FileOutputCommitter{outputPath=file:/Users/stevel/Projects/sparkwork/spark-cloud-examples/cloud-examples/target/tmp/stevel/job_20170323184243_0000/staging-uploads, workPath= null , algorithmVersion=1, skipCleanup= false , ignoreCleanupFailures= true }} for s3a://hwdev-steve- new /spark-cloud/S3ANumbersSuiteV2APISuite/numbers_rdd_tests_v2api 2017-03-23 18:42:44,339 [Executor task launch worker for task 0] INFO commit.DurationInfo (DurationInfo.java:<init>(33)) - Starting Task committer attempt_20170323184242_0000_r_000000_0: task attempt_20170323184242_0000_r_000000_0: creating task attempt path file:/Users/stevel/Projects/sparkwork/spark-cloud-examples/cloud-examples/target/test/mapred/local/job_20170323184243_0000/0/attempt_20170323184242_0000_r_000000_0/_temporary/0/_temporary/attempt_20170323184242_0000_r_000000_0 2017-03-23 18:42:44,386 [Executor task launch worker for task 0] INFO commit.DurationInfo (DurationInfo.java:close(44)) - Task committer attempt_20170323184242_0000_r_000000_0: task attempt_20170323184242_0000_r_000000_0: creating task attempt path file:/Users/stevel/Projects/sparkwork/spark-cloud-examples/cloud-examples/target/test/mapred/local/job_20170323184243_0000/0/attempt_20170323184242_0000_r_000000_0/_temporary/0/_temporary/attempt_20170323184242_0000_r_000000_0 : 0:00:047 2017-03-23 18:42:44,388 [Executor task launch worker for task 0] DEBUG output.PathOutputCommitterFactory (PathOutputCommitterFactory.java:getOutputCommitterFactory(92)) - Using FileOutputCommitter factory class class org.apache.hadoop.fs.s3a.commit.staging.DirectoryStagingCommitterFactory 2017-03-23 18:42:44,388 [Executor task launch worker for task 0] DEBUG commit.AbstractS3GuardCommitter (AbstractS3GuardCommitter.java:<init>(85)) - Task committer attempt_20170323184242_0000_r_000000_0 instantiated for job "" ID job_20170323184242_0000 with destination s3a: //hwdev-steve- new /spark-cloud/S3ANumbersSuiteV2APISuite/numbers_rdd_tests_v2api 2017-03-23 18:42:44,388 [Executor task launch worker for task 0] DEBUG commit.AbstractS3GuardCommitter (AbstractS3GuardCommitter.java:<init>(115)) - Task committer attempt_20170323184242_0000_r_000000_0} instantiated for "" ID job_20170323184242_0000 2017-03-23 18:42:44,389 [Executor task launch worker for task 0] DEBUG commit.AbstractS3GuardCommitter (AbstractS3GuardCommitter.java:setWorkPath(162)) - Setting work path to file:/Users/stevel/Projects/sparkwork/spark-cloud-examples/cloud-examples/target/test/mapred/local/job_20170323184243_0000/0/attempt_20170323184242_0000_r_000000_0/_temporary/0/_temporary/attempt_20170323184242_0000_r_000000_0 2017-03-23 18:42:44,390 [Executor task launch worker for task 0] INFO output.FileOutputCommitter (FileOutputCommitter.java:<init>(123)) - File Output Committer Algorithm version is 1 2017-03-23 18:42:44,390 [Executor task launch worker for task 0] INFO output.FileOutputCommitter (FileOutputCommitter.java:<init>(138)) - FileOutputCommitter skip cleanup _temporary folders under output directory: false , ignore cleanup failures: true 2017-03-23 18:42:44,390 [Executor task launch worker for task 0] DEBUG staging.StagingS3GuardCommitter (StagingS3GuardCommitter.java:postCreationActions(168)) - Conflict resolution mode: FAIL 2017-03-23 18:42:44,390 [Executor task launch worker for task 0] INFO commit.Abstract3GuardCommitterFactory (Abstract3GuardCommitterFactory.java:createOutputCommitter(61)) - Using Commmitter StagingS3GuardCommitter{AbstractS3GuardCommitter{role=Task committer attempt_20170323184242_0000_r_000000_0outputPath=s3a: //hwdev-steve- new /spark-cloud/S3ANumbersSuiteV2APISuite/numbers_rdd_tests_v2api, workPath=file:/Users/stevel/Projects/sparkwork/spark-cloud-examples/cloud-examples/target/test/mapred/local/job_20170323184243_0000/0/attempt_20170323184242_0000_r_000000_0/_temporary/0/_temporary/attempt_20170323184242_0000_r_000000_0}, finalOutputPath= null , conflictResolution=FAIL, wrappedCommitter=FileOutputCommitter{outputPath=file:/Users/stevel/Projects/sparkwork/spark-cloud-examples/cloud-examples/target/tmp/stevel/job_20170323184243_0000/staging-uploads, workPath= null , algorithmVersion=1, skipCleanup= false , ignoreCleanupFailures= true }} for s3a://hwdev-steve- new /spark-cloud/S3ANumbersSuiteV2APISuite/numbers_rdd_tests_v2api 2017-03-23 18:42:44,448 [Executor task launch worker for task 0] INFO commit.DurationInfo (DurationInfo.java:<init>(33)) - Starting Task committer attempt_20170323184242_0000_r_000000_0: needsTaskCommit() Task attempt_20170323184242_0000_r_000000_0 2017-03-23 18:42:44,454 [Executor task launch worker for task 0] DEBUG staging.StagingS3GuardCommitter (StagingS3GuardCommitter.java:needsTaskCommit(849)) - 1 files to commit 2017-03-23 18:42:44,455 [Executor task launch worker for task 0] INFO commit.DurationInfo (DurationInfo.java:close(44)) - Task committer attempt_20170323184242_0000_r_000000_0: needsTaskCommit() Task attempt_20170323184242_0000_r_000000_0: 0:00:006 2017-03-23 18:42:44,457 [Executor task launch worker for task 0] INFO commit.DurationInfo (DurationInfo.java:<init>(33)) - Starting Task committer attempt_20170323184242_0000_r_000000_0: commit task attempt_20170323184242_0000_r_000000_0 2017-03-23 18:42:44,457 [Executor task launch worker for task 0] DEBUG staging.StagingS3GuardCommitter (StagingS3GuardCommitter.java:getTaskOutput(419)) - Scanning file:/Users/stevel/Projects/sparkwork/spark-cloud-examples/cloud-examples/target/test/mapred/local/job_20170323184243_0000/0/attempt_20170323184242_0000_r_000000_0/_temporary/0/_temporary/attempt_20170323184242_0000_r_000000_0 for files to commit 2017-03-23 18:42:44,468 [Executor task launch worker for task 0] DEBUG staging.StagingS3GuardCommitter (StagingS3GuardCommitter.java:commitTaskInternal(885)) - Task committer attempt_20170323184242_0000_r_000000_0: commitTaskInternal 2017-03-23 18:42:44,468 [Executor task launch worker for task 0] DEBUG staging.StagingS3GuardCommitter (StagingS3GuardCommitter.java:commitTaskInternal(890)) - Task committer attempt_20170323184242_0000_r_000000_0: attempt path is file:/Users/stevel/Projects/sparkwork/spark-cloud-examples/cloud-examples/target/test/mapred/local/job_20170323184243_0000/0/attempt_20170323184242_0000_r_000000_0/_temporary/0/_temporary/attempt_20170323184242_0000_r_000000_0 2017-03-23 18:42:44,468 [Executor task launch worker for task 0] INFO staging.StagingS3GuardCommitter (StagingS3GuardCommitter.java:commitTaskInternal(903)) - Task committer attempt_20170323184242_0000_r_000000_0: uploading from staging directory to S3 2017-03-23 18:42:44,468 [Executor task launch worker for task 0] INFO staging.StagingS3GuardCommitter (StagingS3GuardCommitter.java:commitTaskInternal(904)) - Task committer attempt_20170323184242_0000_r_000000_0: Saving pending data information to file:/Users/stevel/Projects/sparkwork/spark-cloud-examples/cloud-examples/target/tmp/stevel/job_20170323184243_0000/staging-uploads/_temporary/0/_temporary/attempt_20170323184242_0000_r_000000_0 2017-03-23 18:42:44,469 [Executor task launch worker for task 0] DEBUG staging.StagingS3GuardCommitter (StagingS3GuardCommitter.java:getOutputPath(1057)) - Task committer attempt_20170323184242_0000_r_000000_0: final output path is s3a: //hwdev-steve- new /spark-cloud/S3ANumbersSuiteV2APISuite/numbers_rdd_tests_v2api 2017-03-23 18:42:44,493 [Executor task launch worker for task 0] DEBUG staging.S3Util (S3Util.java:multipartUpload(155)) - Initiating multipart upload from /Users/stevel/Projects/sparkwork/spark-cloud-examples/cloud-examples/target/test/mapred/local/job_20170323184243_0000/0/attempt_20170323184242_0000_r_000000_0/_temporary/0/_temporary/attempt_20170323184242_0000_r_000000_0/part-r-00000 to s3a: //hwdev-steve- new /spark-cloud/S3ANumbersSuiteV2APISuite/numbers_rdd_tests_v2api/part-r-00000-job_20170323184243_0000 partition= null 2017-03-23 18:42:44,684 [Executor task launch worker for task 0] DEBUG staging.S3Util (S3Util.java:multipartUpload(177)) - File size is 515303, number of parts to upload = 1 2017-03-23 18:42:46,352 [Executor task launch worker for task 0] DEBUG staging.StagingS3GuardCommitter (StagingS3GuardCommitter.java:run(939)) - Task committer attempt_20170323184242_0000_r_000000_0: adding pending commit PendingUpload{ to 's3a: //hwdev-steve- new /spark-cloud/S3ANumbersSuiteV2APISuite/numbers_rdd_tests_v2api/part-r-00000-job_20170323184243_0000' , uploadId='_P7n0N2FxTdVYiTBBFNB2Y9TOQgnbuKT1Lt.mwfx4pdmT6aK4E5XlxfpFZNwSUv0viLzNx45qvhJAqDyhSDqajozcfuFesRF2enq.bpkfc0-'} 2017-03-23 18:42:46,360 [Executor task launch worker for task 0] INFO output.FileOutputCommitter (FileOutputCommitter.java:commitTask(582)) - Saved output of task 'attempt_20170323184242_0000_r_000000_0' to file:/Users/stevel/Projects/sparkwork/spark-cloud-examples/cloud-examples/target/tmp/stevel/job_20170323184243_0000/staging-uploads/_temporary/0/task_20170323184242_0000_r_000000 2017-03-23 18:42:46,362 [Executor task launch worker for task 0] INFO staging.StagingS3GuardCommitter (StagingS3GuardCommitter.java:commitTask(866)) - Task committer attempt_20170323184242_0000_r_000000_0: upload file count: 1 2017-03-23 18:42:46,362 [Executor task launch worker for task 0] INFO commit.DurationInfo (DurationInfo.java:close(44)) - Task committer attempt_20170323184242_0000_r_000000_0: commit task attempt_20170323184242_0000_r_000000_0: 0:01:906 2017-03-23 18:42:46,363 [Executor task launch worker for task 0] INFO mapred.SparkHadoopMapRedUtil (Logging.scala:logInfo(54)) - attempt_20170323184242_0000_r_000000_0: Committed 2017-03-23 18:42:46,374 [Executor task launch worker for task 0] INFO executor.Executor (Logging.scala:logInfo(54)) - Finished task 0.0 in stage 0.0 (TID 0). 1306 bytes result sent to driver 2017-03-23 18:42:46,380 [task-result-getter-0] INFO scheduler.TaskSetManager (Logging.scala:logInfo(54)) - Finished task 0.0 in stage 0.0 (TID 0) in 2111 ms on localhost (executor driver) (1/1) 2017-03-23 18:42:46,381 [task-result-getter-0] INFO scheduler.TaskSchedulerImpl (Logging.scala:logInfo(54)) - Removed TaskSet 0.0, whose tasks have all completed, from pool 2017-03-23 18:42:46,384 [dag-scheduler-event-loop] INFO scheduler.DAGScheduler (Logging.scala:logInfo(54)) - ResultStage 0 (runJob at SparkHadoopMapReduceWriter.scala:88) finished in 2.124 s 2017-03-23 18:42:46,388 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] INFO scheduler.DAGScheduler (Logging.scala:logInfo(54)) - Job 0 finished: runJob at SparkHadoopMapReduceWriter.scala:88, took 2.265104 s 2017-03-23 18:42:46,389 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] INFO commit.DurationInfo (DurationInfo.java:<init>(33)) - Starting Task committer attempt_20170323184243_0000_m_000000_0: preparing to commit Job 2017-03-23 18:42:46,390 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] DEBUG staging.StagingS3GuardCommitter (StagingS3GuardCommitter.java:getThreadPool(1094)) - Task committer attempt_20170323184243_0000_m_000000_0: creating thread pool of size 8 2017-03-23 18:42:46,393 [s3-committer-pool-0] DEBUG staging.S3Util (S3Util.java:readPendingCommits(234)) - Reading pending commits in file file:/Users/stevel/Projects/sparkwork/spark-cloud-examples/cloud-examples/target/tmp/stevel/job_20170323184243_0000/staging-uploads/_temporary/0/task_20170323184242_0000_r_000000 2017-03-23 18:42:46,406 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] DEBUG s3a.S3AFileSystem (S3AFileSystem.java:innerGetFileStatus(1782)) - Getting path status for s3a: //hwdev-steve- new /spark-cloud/S3ANumbersSuiteV2APISuite/numbers_rdd_tests_v2api (spark-cloud/S3ANumbersSuiteV2APISuite/numbers_rdd_tests_v2api) 2017-03-23 18:42:47,006 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] DEBUG s3a.S3AFileSystem (S3AFileSystem.java:s3GetFileStatus(1905)) - Not Found: s3a: //hwdev-steve- new /spark-cloud/S3ANumbersSuiteV2APISuite/numbers_rdd_tests_v2api 2017-03-23 18:42:47,006 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] INFO commit.DurationInfo (DurationInfo.java:close(44)) - Task committer attempt_20170323184243_0000_m_000000_0: preparing to commit Job: 0:00:617 2017-03-23 18:42:47,007 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] INFO commit.DurationInfo (DurationInfo.java:<init>(33)) - Starting Task committer attempt_20170323184243_0000_m_000000_0: committing Job job_20170323184242_0001 2017-03-23 18:42:47,007 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] DEBUG staging.StagingS3GuardCommitter (StagingS3GuardCommitter.java:commitJobInternal(649)) - Task committer attempt_20170323184243_0000_m_000000_0: committing the output of 1 task(s) 2017-03-23 18:42:47,010 [s3-committer-pool-1] DEBUG staging.S3Util (S3Util.java:finishCommit(101)) - Finish PendingUpload{ to 's3a: //hwdev-steve- new /spark-cloud/S3ANumbersSuiteV2APISuite/numbers_rdd_tests_v2api/part-r-00000-job_20170323184243_0000' , uploadId='_P7n0N2FxTdVYiTBBFNB2Y9TOQgnbuKT1Lt.mwfx4pdmT6aK4E5XlxfpFZNwSUv0viLzNx45qvhJAqDyhSDqajozcfuFesRF2enq.bpkfc0-'} 2017-03-23 18:42:47,364 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] DEBUG s3a.S3AFileSystem (S3AFileSystem.java:innerGetFileStatus(1782)) - Getting path status for s3a: //hwdev-steve- new /spark-cloud/S3ANumbersSuiteV2APISuite/numbers_rdd_tests_v2api/_temporary (spark-cloud/S3ANumbersSuiteV2APISuite/numbers_rdd_tests_v2api/_temporary) 2017-03-23 18:42:47,941 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] DEBUG s3a.S3AFileSystem (S3AFileSystem.java:s3GetFileStatus(1905)) - Not Found: s3a: //hwdev-steve- new /spark-cloud/S3ANumbersSuiteV2APISuite/numbers_rdd_tests_v2api/_temporary 2017-03-23 18:42:47,942 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] DEBUG s3a.S3AFileSystem (S3AFileSystem.java:delete(1418)) - Couldn't delete s3a: //hwdev-steve- new /spark-cloud/S3ANumbersSuiteV2APISuite/numbers_rdd_tests_v2api/_temporary - does not exist 2017-03-23 18:42:47,943 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] INFO commit.DurationInfo (DurationInfo.java:close(44)) - Task committer attempt_20170323184243_0000_m_000000_0: committing Job job_20170323184242_0001: 0:00:936 2017-03-23 18:42:47,943 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] DEBUG commit.FileCommitActions (FileCommitActions.java:touchSuccessMarker(231)) - Touching success marker for job s3a: //hwdev-steve- new /spark-cloud/S3ANumbersSuiteV2APISuite/numbers_rdd_tests_v2api/_SUCCESS 2017-03-23 18:42:47,944 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] DEBUG s3a.S3AFileSystem (S3AFileSystem.java:innerGetFileStatus(1782)) - Getting path status for s3a: //hwdev-steve- new /spark-cloud/S3ANumbersSuiteV2APISuite/numbers_rdd_tests_v2api/_SUCCESS (spark-cloud/S3ANumbersSuiteV2APISuite/numbers_rdd_tests_v2api/_SUCCESS) 2017-03-23 18:42:48,494 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] DEBUG s3a.S3AFileSystem (S3AFileSystem.java:s3GetFileStatus(1905)) - Not Found: s3a: //hwdev-steve- new /spark-cloud/S3ANumbersSuiteV2APISuite/numbers_rdd_tests_v2api/_SUCCESS 2017-03-23 18:42:48,495 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] DEBUG commit.DelayedCommitFSIntegration (DelayedCommitFSIntegration.java:getTracker(98)) - Created DefaultPutTracker{destKey='spark-cloud/S3ANumbersSuiteV2APISuite/numbers_rdd_tests_v2api/_SUCCESS'} 2017-03-23 18:42:48,498 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] DEBUG s3a.S3AFileSystem (S3AFileSystem.java:createWriteOperationHelper(703)) - Created WriteOperationHelper {bucket=hwdev-steve- new , key='spark-cloud/S3ANumbersSuiteV2APISuite/numbers_rdd_tests_v2api/_SUCCESS'} 2017-03-23 18:42:48,514 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] DEBUG s3a.S3ADataBlocks (S3ADataBlocks.java:startUpload(324)) - Start datablock[1] upload 2017-03-23 18:42:48,514 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] DEBUG s3a.S3ADataBlocks (S3ADataBlocks.java:enterState(231)) - FileBlock{index=1, destFile=/Users/stevel/Projects/sparkwork/spark-cloud-examples/cloud-examples/target/tmp/s3ablock-0001-4746647579488981641.tmp, state=Writing, dataSize=0, limit=8388608}: entering state Upload 2017-03-23 18:42:48,518 [s3a-transfer-shared-pool1-t1] DEBUG s3a.S3AFileSystem (S3AFileSystem.java:putObjectDirect(1280)) - PUT 0 bytes to spark-cloud/S3ANumbersSuiteV2APISuite/numbers_rdd_tests_v2api/_SUCCESS 2017-03-23 18:42:48,519 [s3a-transfer-shared-pool1-t1] DEBUG s3a.S3AFileSystem (S3AFileSystem.java:incrementPutStartStatistics(1321)) - PUT start 0 bytes 2017-03-23 18:42:48,740 [s3a-transfer-shared-pool1-t1] DEBUG s3a.S3AFileSystem (S3AFileSystem.java:incrementPutCompletedStatistics(1338)) - PUT completed success= true ; 0 bytes 2017-03-23 18:42:48,740 [s3a-transfer-shared-pool1-t1] DEBUG s3a.S3AFileSystem (S3AFileSystem.java:finishedWrite(2183)) - Finished write to spark-cloud/S3ANumbersSuiteV2APISuite/numbers_rdd_tests_v2api/_SUCCESS, len 0 2017-03-23 18:42:48,938 [s3a-transfer-shared-pool1-t1] DEBUG s3a.S3ADataBlocks (S3ADataBlocks.java:enterState(231)) - FileBlock{index=1, destFile=/Users/stevel/Projects/sparkwork/spark-cloud-examples/cloud-examples/target/tmp/s3ablock-0001-4746647579488981641.tmp, state=Upload, dataSize=0, limit=8388608}: entering state Closed 2017-03-23 18:42:48,938 [s3a-transfer-shared-pool1-t1] DEBUG s3a.S3ADataBlocks (S3ADataBlocks.java:close(346)) - Closed FileBlock{index=1, destFile=/Users/stevel/Projects/sparkwork/spark-cloud-examples/cloud-examples/target/tmp/s3ablock-0001-4746647579488981641.tmp, state=Closed, dataSize=0, limit=8388608} 2017-03-23 18:42:48,938 [s3a-transfer-shared-pool1-t1] DEBUG s3a.S3ADataBlocks (S3ADataBlocks.java:innerClose(886)) - Closing FileBlock{index=1, destFile=/Users/stevel/Projects/sparkwork/spark-cloud-examples/cloud-examples/target/tmp/s3ablock-0001-4746647579488981641.tmp, state=Closed, dataSize=0, limit=8388608} 2017-03-23 18:42:48,939 [s3a-transfer-shared-pool1-t1] DEBUG s3a.S3ADataBlocks (S3ADataBlocks.java:closeBlock(940)) - block[1]: closeBlock() 2017-03-23 18:42:48,939 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] DEBUG s3a.S3AFileSystem (S3AFileSystem.java:writeSuccessful(2701)) - Successful write to spark-cloud/S3ANumbersSuiteV2APISuite/numbers_rdd_tests_v2api/_SUCCESS, len 0 2017-03-23 18:42:48,941 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] DEBUG s3a.S3AFileSystem (S3AFileSystem.java:innerGetFileStatus(1782)) - Getting path status for s3a: //hwdev-steve- new /spark-cloud/S3ANumbersSuiteV2APISuite/numbers_rdd_tests_v2api/_temporary-1 (spark-cloud/S3ANumbersSuiteV2APISuite/numbers_rdd_tests_v2api/_temporary-1) 2017-03-23 18:42:50,072 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] DEBUG s3a.S3AFileSystem (S3AFileSystem.java:s3GetFileStatus(1905)) - Not Found: s3a: //hwdev-steve- new /spark-cloud/S3ANumbersSuiteV2APISuite/numbers_rdd_tests_v2api/_temporary-1 2017-03-23 18:42:50,072 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] DEBUG s3a.S3AFileSystem (S3AFileSystem.java:delete(1418)) - Couldn't delete s3a: //hwdev-steve- new /spark-cloud/S3ANumbersSuiteV2APISuite/numbers_rdd_tests_v2api/_temporary-1 - does not exist 2017-03-23 18:42:50,074 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] INFO io.SparkHadoopMapReduceWriter (Logging.scala:logInfo(54)) - Job job_20170323184242_0001 committed. 2017-03-23 18:42:50,081 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] INFO ui.SparkUI (Logging.scala:logInfo(54)) - Stopped Spark web UI at http: //192.168.1.38:4040 2017-03-23 18:42:50,108 [dispatcher-event-loop-2] INFO spark.MapOutputTrackerMasterEndpoint (Logging.scala:logInfo(54)) - MapOutputTrackerMasterEndpoint stopped! 2017-03-23 18:42:50,117 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] INFO memory.MemoryStore (Logging.scala:logInfo(54)) - MemoryStore cleared 2017-03-23 18:42:50,118 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] INFO storage.BlockManager (Logging.scala:logInfo(54)) - BlockManager stopped 2017-03-23 18:42:50,123 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] INFO storage.BlockManagerMaster (Logging.scala:logInfo(54)) - BlockManagerMaster stopped 2017-03-23 18:42:50,125 [dispatcher-event-loop-0] INFO scheduler.OutputCommitCoordinator$OutputCommitCoordinatorEndpoint (Logging.scala:logInfo(54)) - OutputCommitCoordinator stopped! 2017-03-23 18:42:50,128 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] INFO spark.SparkContext (Logging.scala:logInfo(54)) - Successfully stopped SparkContext 2017-03-23 18:42:50,130 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] INFO s3.S3ANumbersSuiteV2APISuite (CloudLogging.scala:logInfo(56)) - Filesystem statistics S3AFileSystem {uri=s3a: //hwdev-steve- new , workingDir=s3a://hwdev-steve- new /user/stevel, inputPolicy=sequential, partSize=8388608, enableMultiObjectsDelete= true , maxKeys=5000, readAhead=65536, blockSize=1048576, multiPartThreshold=2147483647, serverSideEncryptionAlgorithm='NONE', blockFactory=org.apache.hadoop.fs.s3a.S3ADataBlocks$DiskBlockFactory@6707a4bf, metastore=NullMetadataStore, authoritative= false , boundedExecutor=BlockingThreadPoolExecutorService {SemaphoredDelegatingExecutor {permitCount=25, available=25, waiting=0}, activeCount=0}, unboundedExecutor=java.util.concurrent.ThreadPoolExecutor@3fc7c734[Running, pool size = 0, active threads = 0, queued tasks = 0, completed tasks = 0], statistics {0 bytes read, 0 bytes written, 7 read ops, 0 large read ops, 3 write ops}, metrics { {Context=S3AFileSystem} {FileSystemId=58e091c4-0ae9-4626-8ecf-0d1ac3ffdb27-hwdev-steve- new } {fsURI=s3a: //hwdev-steve- new } {files_created=1} {files_copied=0} {files_copied_bytes=0} {files_deleted=0} {fake_directories_deleted=3} {directories_created=0} {directories_deleted=0} {ignored_errors=3} {op_copy_from_local_file=0} {op_exists=3} {op_get_file_status=7} {op_glob_status=0} {op_is_directory=0} {op_is_file=0} {op_list_files=0} {op_list_located_status=0} {op_list_status=0} {op_mkdirs=0} {op_rename=0} {object_copy_requests=0} {object_delete_requests=1} {object_list_requests=7} {object_continue_list_requests=0} {object_metadata_requests=14} {object_multipart_aborted=0} {object_put_bytes=0} {object_put_requests=1} {object_put_requests_completed=1} {stream_write_failures=0} {stream_write_block_uploads=0} {stream_write_block_uploads_committed=0} {stream_write_block_uploads_aborted=0} {stream_write_total_time=0} {stream_write_total_data=0} {object_put_requests_active=0} {object_put_bytes_pending=0} {stream_write_block_uploads_active=0} {stream_write_block_uploads_pending=1} {stream_write_block_uploads_data_pending=0} {stream_read_fully_operations=0} {stream_opened=0} {stream_bytes_skipped_on_seek=0} {stream_closed=0} {stream_bytes_backwards_on_seek=0} {stream_bytes_read=0} {stream_read_operations_incomplete=0} {stream_bytes_discarded_in_abort=0} {stream_close_operations=0} {stream_read_operations=0} {stream_aborted=0} {stream_forward_seek_operations=0} {stream_backward_seek_operations=0} {stream_seek_operations=0} {stream_bytes_read_in_close=0} {stream_read_exceptions=0} }} 2017-03-23 18:42:50,130 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] DEBUG s3a.S3AFileSystem (S3AFileSystem.java:listFiles(2505)) - listFiles(s3a: //hwdev-steve- new /spark-cloud/S3ANumbersSuiteV2APISuite/numbers_rdd_tests_v2api, true ) 2017-03-23 18:42:50,130 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] DEBUG s3a.S3AFileSystem (S3AFileSystem.java:innerGetFileStatus(1782)) - Getting path status for s3a: //hwdev-steve- new /spark-cloud/S3ANumbersSuiteV2APISuite/numbers_rdd_tests_v2api (spark-cloud/S3ANumbersSuiteV2APISuite/numbers_rdd_tests_v2api) 2017-03-23 18:42:50,761 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] DEBUG s3a.S3AFileSystem (S3AFileSystem.java:s3GetFileStatus(1880)) - Found path as directory (with /): 0/1 2017-03-23 18:42:50,761 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] DEBUG s3a.S3AFileSystem (S3AFileSystem.java:s3GetFileStatus(1885)) - Summary: spark-cloud/S3ANumbersSuiteV2APISuite/numbers_rdd_tests_v2api/_SUCCESS 0 2017-03-23 18:42:50,763 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] DEBUG s3a.S3AFileSystem (S3AFileSystem.java:listFiles(2518)) - Requesting all entries under spark-cloud/S3ANumbersSuiteV2APISuite/numbers_rdd_tests_v2api/ with delimiter ' null ' 2017-03-23 18:42:50,953 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] DEBUG s3a.S3AFileSystem (Listing.java:buildNextStatusBatch(296)) - s3a: //hwdev-steve- new /spark-cloud/S3ANumbersSuiteV2APISuite/numbers_rdd_tests_v2api/_SUCCESS: spark-cloud/S3ANumbersSuiteV2APISuite/numbers_rdd_tests_v2api/_SUCCESS size=0 2017-03-23 18:42:50,954 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] DEBUG s3a.S3AFileSystem (Listing.java:buildNextStatusBatch(302)) - Adding: S3AFileStatus{path=s3a: //hwdev-steve- new /spark-cloud/S3ANumbersSuiteV2APISuite/numbers_rdd_tests_v2api/_SUCCESS; isDirectory= false ; length=0; replication=1; blocksize=1048576; modification_time=1490294569000; access_time=0; owner=stevel; group=stevel; permission=rw-rw-rw-; isSymlink= false } isEmptyDirectory=FALSE 2017-03-23 18:42:50,954 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] DEBUG s3a.S3AFileSystem (Listing.java:buildNextStatusBatch(296)) - s3a: //hwdev-steve- new /spark-cloud/S3ANumbersSuiteV2APISuite/numbers_rdd_tests_v2api/part-r-00000-job_20170323184243_0000: spark-cloud/S3ANumbersSuiteV2APISuite/numbers_rdd_tests_v2api/part-r-00000-job_20170323184243_0000 size=515303 2017-03-23 18:42:50,955 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] DEBUG s3a.S3AFileSystem (Listing.java:buildNextStatusBatch(302)) - Adding: S3AFileStatus{path=s3a: //hwdev-steve- new /spark-cloud/S3ANumbersSuiteV2APISuite/numbers_rdd_tests_v2api/part-r-00000-job_20170323184243_0000; isDirectory= false ; length=515303; replication=1; blocksize=1048576; modification_time=1490294565000; access_time=0; owner=stevel; group=stevel; permission=rw-rw-rw-; isSymlink= false } isEmptyDirectory=FALSE 2017-03-23 18:42:50,955 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] DEBUG s3a.S3AFileSystem (Listing.java:buildNextStatusBatch(330)) - Added 2 entries; ignored 0; hasNext= true ; hasMoreObjects= false 2017-03-23 18:42:50,957 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] INFO s3.S3ANumbersSuiteV2APISuite (CloudLogging.scala:logInfo(56)) - Contents of s3a: //hwdev-steve- new /spark-cloud/S3ANumbersSuiteV2APISuite/numbers_rdd_tests_v2api: s3a: //hwdev-steve- new /spark-cloud/S3ANumbersSuiteV2APISuite/numbers_rdd_tests_v2api/_SUCCESS [0] s3a: //hwdev-steve- new /spark-cloud/S3ANumbersSuiteV2APISuite/numbers_rdd_tests_v2api/part-r-00000-job_20170323184243_0000 [515303] Dirs: 0, Files: 2, total bytes = 515303 2017-03-23 18:42:50,964 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] INFO spark.SparkContext (Logging.scala:logInfo(54)) - SparkContext already stopped. 2017-03-23 18:42:50,966 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] DEBUG s3a.S3AFileSystem (S3AFileSystem.java:innerGetFileStatus(1782)) - Getting path status for s3a: //hwdev-steve- new /spark-cloud/S3ANumbersSuiteV2APISuite (spark-cloud/S3ANumbersSuiteV2APISuite) - SaveRDD + Cleaning s3a: //hwdev-steve- new /spark-cloud/S3ANumbersSuiteV2APISuite 2017-03-23 18:42:51,514 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] DEBUG s3a.S3AFileSystem (S3AFileSystem.java:s3GetFileStatus(1880)) - Found path as directory (with /): 1/0 2017-03-23 18:42:51,514 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] DEBUG s3a.S3AFileSystem (S3AFileSystem.java:s3GetFileStatus(1888)) - Prefix: spark-cloud/S3ANumbersSuiteV2APISuite/numbers_rdd_tests_v2api/ 2017-03-23 18:42:51,514 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] DEBUG s3a.S3AFileSystem (S3AFileSystem.java:innerGetFileStatus(1782)) - Getting path status for s3a: //hwdev-steve- new /spark-cloud/S3ANumbersSuiteV2APISuite (spark-cloud/S3ANumbersSuiteV2APISuite) 2017-03-23 18:42:52,059 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] DEBUG s3a.S3AFileSystem (S3AFileSystem.java:s3GetFileStatus(1880)) - Found path as directory (with /): 1/0 2017-03-23 18:42:52,059 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] DEBUG s3a.S3AFileSystem (S3AFileSystem.java:s3GetFileStatus(1888)) - Prefix: spark-cloud/S3ANumbersSuiteV2APISuite/numbers_rdd_tests_v2api/ 2017-03-23 18:42:52,060 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] DEBUG s3a.S3AFileSystem (S3AFileSystem.java:innerDelete(1444)) - Delete path s3a: //hwdev-steve- new /spark-cloud/S3ANumbersSuiteV2APISuite - recursive true 2017-03-23 18:42:52,060 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] DEBUG s3a.S3AFileSystem (S3AFileSystem.java:innerDelete(1449)) - delete: Path is a directory: s3a: //hwdev-steve- new /spark-cloud/S3ANumbersSuiteV2APISuite 2017-03-23 18:42:52,060 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] DEBUG s3a.S3AFileSystem (S3AFileSystem.java:innerDelete(1473)) - Getting objects for directory prefix spark-cloud/S3ANumbersSuiteV2APISuite/ to delete 2017-03-23 18:42:52,248 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] DEBUG s3a.S3AFileSystem (S3AFileSystem.java:innerDelete(1483)) - Got object to delete spark-cloud/S3ANumbersSuiteV2APISuite/numbers_rdd_tests_v2api/_SUCCESS 2017-03-23 18:42:52,248 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] DEBUG s3a.S3AFileSystem (S3AFileSystem.java:innerDelete(1483)) - Got object to delete spark-cloud/S3ANumbersSuiteV2APISuite/numbers_rdd_tests_v2api/part-r-00000-job_20170323184243_0000 2017-03-23 18:42:53,499 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] DEBUG s3a.S3AFileSystem (S3AFileSystem.java:s3GetFileStatus(1880)) - Found path as directory (with /): 1/0 2017-03-23 18:42:53,500 [ScalaTest-main-running-S3ANumbersSuiteV2APISuite] DEBUG s3a.S3AFileSystem (S3AFileSystem.java:s3GetFileStatus(1888)) - Prefix: spark-cloud/S3ALineCountWritebackSuite/ Run completed in 15 seconds, 386 milliseconds.
          Hide
          stevel@apache.org Steve Loughran added a comment -

          patch #20: test changes; nothing to the production code

          • lambdas -> anonymous classes so things would backport to branch-2
          • fault injection class created for ability to fail any of the IT test operations, see what happens in the protocols.
          • IT test of MR job, AbstractITCommitMRJob based on the mock one: sharing core mini cluster setup, test case modified to look for final data. Currently just staging, possible for the others too. Twill need separate assertions. Maybe this can just be parameterized.
          • StorageStatisticsTracker to snapshot/diff storage stats. I'd hoped to use this for tracking IO in MR job, but as MR doesn't collect storage stats, not yet used. It should be useful in other tests though.

          The new IT MR test works, once I've enabled the -unique-filenames option in the test. I've also turned on the same switch for the protocol tests: That doesn't work for the mapper tests.

          java.io.FileNotFoundException: index file in s3a://hwdev-steve-ireland-new/test/ITestDirectoryCommitProtocol-testMapFileOutputCommitter/part-m-00000: not found s3a://hwdev-steve-ireland-new/test/ITestDirectoryCommitProtocol-testMapFileOutputCommitter/part-m-00000/index in s3a://hwdev-steve-ireland-new/test/ITestDirectoryCommitProtocol-testMapFileOutputCommitter/part-m-00000
          
          	at org.apache.hadoop.fs.contract.ContractTestUtils.verifyPathExists(ContractTestUtils.java:779)
          	at org.apache.hadoop.fs.contract.ContractTestUtils.assertPathExists(ContractTestUtils.java:757)
          	at org.apache.hadoop.fs.contract.AbstractFSContractTestBase.assertPathExists(AbstractFSContractTestBase.java:294)
          	at org.apache.hadoop.fs.s3a.commit.AbstractITCommitProtocol.validateMapFileOutputContent(AbstractITCommitProtocol.java:607)
          	at org.apache.hadoop.fs.s3a.commit.AbstractITCommitProtocol.testMapFileOutputCommitter(AbstractITCommitProtocol.java:947)
          	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.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:47)
          	at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12)
          	at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:44)
          	at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17)
          	at org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26)
          	at org.junit.internal.runners.statements.RunAfters.evaluate(RunAfters.java:27)
          	at org.junit.rules.TestWatcher$1.evaluate(TestWatcher.java:55)
          	at org.junit.internal.runners.statements.FailOnTimeout$StatementThread.run(FailOnTimeout.java:74)
          Caused by: java.io.FileNotFoundException: No such file or directory: s3a://hwdev-steve-ireland-new/test/ITestDirectoryCommitProtocol-testMapFileOutputCommitter/part-m-00000/index
          	at org.apache.hadoop.fs.s3a.S3AFileSystem.s3GetFileStatus(S3AFileSystem.java:1906)
          	at org.apache.hadoop.fs.s3a.S3AFileSystem.innerGetFileStatus(S3AFileSystem.java:1802)
          	at org.apache.hadoop.fs.s3a.S3AFileSystem.getFileStatus(S3AFileSystem.java:1764)
          	at org.apache.hadoop.fs.contract.ContractTestUtils.verifyPathExists(ContractTestUtils.java:773)
          
          Show
          stevel@apache.org Steve Loughran added a comment - patch #20: test changes; nothing to the production code lambdas -> anonymous classes so things would backport to branch-2 fault injection class created for ability to fail any of the IT test operations, see what happens in the protocols. IT test of MR job, AbstractITCommitMRJob based on the mock one: sharing core mini cluster setup, test case modified to look for final data. Currently just staging, possible for the others too. Twill need separate assertions. Maybe this can just be parameterized. StorageStatisticsTracker to snapshot/diff storage stats. I'd hoped to use this for tracking IO in MR job, but as MR doesn't collect storage stats, not yet used. It should be useful in other tests though. The new IT MR test works, once I've enabled the -unique-filenames option in the test. I've also turned on the same switch for the protocol tests: That doesn't work for the mapper tests. java.io.FileNotFoundException: index file in s3a: //hwdev-steve-ireland- new /test/ITestDirectoryCommitProtocol-testMapFileOutputCommitter/part-m-00000: not found s3a://hwdev-steve-ireland- new /test/ITestDirectoryCommitProtocol-testMapFileOutputCommitter/part-m-00000/index in s3a://hwdev-steve-ireland- new /test/ITestDirectoryCommitProtocol-testMapFileOutputCommitter/part-m-00000 at org.apache.hadoop.fs.contract.ContractTestUtils.verifyPathExists(ContractTestUtils.java:779) at org.apache.hadoop.fs.contract.ContractTestUtils.assertPathExists(ContractTestUtils.java:757) at org.apache.hadoop.fs.contract.AbstractFSContractTestBase.assertPathExists(AbstractFSContractTestBase.java:294) at org.apache.hadoop.fs.s3a.commit.AbstractITCommitProtocol.validateMapFileOutputContent(AbstractITCommitProtocol.java:607) at org.apache.hadoop.fs.s3a.commit.AbstractITCommitProtocol.testMapFileOutputCommitter(AbstractITCommitProtocol.java:947) 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.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:47) at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12) at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:44) at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17) at org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26) at org.junit.internal.runners.statements.RunAfters.evaluate(RunAfters.java:27) at org.junit.rules.TestWatcher$1.evaluate(TestWatcher.java:55) at org.junit.internal.runners.statements.FailOnTimeout$StatementThread.run(FailOnTimeout.java:74) Caused by: java.io.FileNotFoundException: No such file or directory: s3a: //hwdev-steve-ireland- new /test/ITestDirectoryCommitProtocol-testMapFileOutputCommitter/part-m-00000/index at org.apache.hadoop.fs.s3a.S3AFileSystem.s3GetFileStatus(S3AFileSystem.java:1906) at org.apache.hadoop.fs.s3a.S3AFileSystem.innerGetFileStatus(S3AFileSystem.java:1802) at org.apache.hadoop.fs.s3a.S3AFileSystem.getFileStatus(S3AFileSystem.java:1764) at org.apache.hadoop.fs.contract.ContractTestUtils.verifyPathExists(ContractTestUtils.java:773)
          Hide
          stevel@apache.org Steve Loughran added a comment -

          side issue: code uses the traditional part-m-%04d naming pattern. Given that s3 uses the first few chars for its hashing, would it better to have a pattern like "p-%04d", or, to be really daring, put the UUID first. I don't think you'd see the benefits of this unless the output was [almost at the root of the bucket though

          https://aws.amazon.com/blogs/aws/amazon-s3-performance-tips-tricks-seattle-hiring-event/]
          Show
          stevel@apache.org Steve Loughran added a comment - side issue: code uses the traditional part-m-%04d naming pattern. Given that s3 uses the first few chars for its hashing, would it better to have a pattern like "p-%04d", or, to be really daring, put the UUID first. I don't think you'd see the benefits of this unless the output was [almost at the root of the bucket though https://aws.amazon.com/blogs/aws/amazon-s3-performance-tips-tricks-seattle-hiring-event/]
          Hide
          stevel@apache.org Steve Loughran added a comment -

          OK, scheduled a meetup at 10:00 PST, 18:00 BST: https://hortonworks.webex.com/hortonworks/j.php?MTID=mfb8dcba47d84a7f67ee9f72872e47cb4 ; I think that URL should be sufficient to join.

          Show
          stevel@apache.org Steve Loughran added a comment - OK, scheduled a meetup at 10:00 PST, 18:00 BST: https://hortonworks.webex.com/hortonworks/j.php?MTID=mfb8dcba47d84a7f67ee9f72872e47cb4 ; I think that URL should be sufficient to join.
          Hide
          stevel@apache.org Steve Loughran added a comment -

          HADOOP-13786 patch 021: unifying data structures and s3 client use

          This iteration is starting to move towards using the S3AFS writeOoperations, with the first step being: unified JSON persistence across alll committers

          • New serializable type, MultiplePendingCommits , containing a list of SinglePendingCommit instances; the latter being the things the magic committer saves.
          • SinglePendingCommit adds fields/operations needed by staging committer. It's a bit ungainly right now; view as an interim step.
          • Staging committers and tests all moved over to this datatype instead of seralized java stream. Good: debugging, security, validation logic. Bad: JSON serialization overhead.
          • in the move, switches the various lists that the thread-pooled staging code buids up to being synchronized lists. I think there may have been risk of race conditions there.

          Other changes

          • default unique name option == false
          • tests can handle option of unique vs non-unique filenames
          • and the partition committer skips the Mapper test. Doesn't make sense.

          Essentinally: the unique name algorithm doesn't work withy map tasks, as they expect a part-m-0000/ dir with children explicitly named "index" and "data". Adding unique names under index and data breaks this.

          I'm still undecided about what the best default value is here, more insight and experimentation needed

          Show
          stevel@apache.org Steve Loughran added a comment - HADOOP-13786 patch 021: unifying data structures and s3 client use This iteration is starting to move towards using the S3AFS writeOoperations, with the first step being: unified JSON persistence across alll committers New serializable type, MultiplePendingCommits , containing a list of SinglePendingCommit instances; the latter being the things the magic committer saves. SinglePendingCommit adds fields/operations needed by staging committer. It's a bit ungainly right now; view as an interim step. Staging committers and tests all moved over to this datatype instead of seralized java stream. Good: debugging, security, validation logic. Bad: JSON serialization overhead. in the move, switches the various lists that the thread-pooled staging code buids up to being synchronized lists. I think there may have been risk of race conditions there. Other changes default unique name option == false tests can handle option of unique vs non-unique filenames and the partition committer skips the Mapper test. Doesn't make sense. Essentinally: the unique name algorithm doesn't work withy map tasks, as they expect a part-m-0000/ dir with children explicitly named "index" and "data". Adding unique names under index and data breaks this. I'm still undecided about what the best default value is here, more insight and experimentation needed
          Hide
          stevel@apache.org Steve Loughran added a comment -

          IT Test testMapFileOutputCommitter failures with unique IDs enabled

          This is failing as the unique ID goes onto the path such as dest/part-m-0000/index-job-attempt_01, part-m-000/data-job-attempt_01. As a result, the files index and data
          aren't found in the test or the reducer. The naming needs to go into the part-m-0000-$UUID/index and part-m-0000-$UUID/index.

          I've disabled unique filenames by default; adapted files to match

          for the partitioned committer, it doesn't work with mappers anyway, because of how things are handled in the commit (as far as I can tell, though I could be confused here.). Disabled the relevant IT test.

          TestStatingMRJob Failures

          Reasons for failure in Yetus unclear. Locally, appears to work, but in the AM log there's a stack warning that s3a isn't bonding

          2017-03-24 16:56:37,468 WARN [CommitterEvent Processor #1] org.apache.hadoop.fs.s3a.S3AUtils: Failed to delete s3a://bucket-name/output/path/_temporary
          java.nio.file.AccessDeniedException: s3a://bucket-name/output/path/_temporary: getFileStatus on s3a://bucket-name/output/path/_temporary: com.amazonaws.services.s3.model.AmazonS3Exception: Forbidden (Service: Amazon S3; Status Code: 403; Error Code: 403 Forbidden; Request ID: 5F7696A22A66FAAF), S3 Extended Request ID: cVVOEgwybZL0YQbQ0fwFbPMv7GO/TOBAVJed//EQsQUX9dGNkgzCrLy1U6ZbMHmb9qQyKzZSOcw=
          	at org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:159)
          	at org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:102)
          	at org.apache.hadoop.fs.s3a.S3AFileSystem.s3GetFileStatus(S3AFileSystem.java:1832)
          	at org.apache.hadoop.fs.s3a.S3AFileSystem.innerGetFileStatus(S3AFileSystem.java:1802)
          	at org.apache.hadoop.fs.s3a.S3AFileSystem.delete(S3AFileSystem.java:1416)
          	at org.apache.hadoop.fs.s3a.S3AUtils.deleteWithWarning(S3AUtils.java:756)
          	at org.apache.hadoop.fs.s3a.commit.staging.StagingS3GuardCommitter.deleteDestinationPaths(StagingS3GuardCommitter.java:817)
          	at org.apache.hadoop.fs.s3a.commit.staging.StagingS3GuardCommitter.cleanup(StagingS3GuardCommitter.java:780)
          	at org.apache.hadoop.fs.s3a.commit.staging.StagingS3GuardCommitter.commitJobInternal(StagingS3GuardCommitter.java:685)
          	at org.apache.hadoop.fs.s3a.commit.staging.StagingS3GuardCommitter.commitJob(StagingS3GuardCommitter.java:625)
          	at org.apache.hadoop.fs.s3a.commit.staging.MockedStagingCommitter.commitJob(MockedStagingCommitter.java:84)
          	at org.apache.hadoop.mapreduce.v2.app.commit.CommitterEventHandler$EventProcessor.handleJobCommit(CommitterEventHandler.java:286)
          	at org.apache.hadoop.mapreduce.v2.app.commit.CommitterEventHandler$EventProcessor.run(CommitterEventHandler.java:238)
          	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
          	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
          	at java.lang.Thread.run(Thread.java:745)
          Caused by: com.amazonaws.services.s3.model.AmazonS3Exception: Forbidden (Service: Amazon S3; Status Code: 403; Error Code: 403 Forbidden; Request ID: 5F7696A22A66FAAF), S3 Extended Request ID: cVVOEgwybZL0YQbQ0fwFbPMv7GO/TOBAVJed//EQsQUX9dGNkgzCrLy1U6ZbMHmb9qQyKzZSOcw=
          	at com.amazonaws.http.AmazonHttpClient$RequestExecutor.handleErrorResponse(AmazonHttpClient.java:1586)
          	at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeOneRequest(AmazonHttpClient.java:1254)
          	at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeHelper(AmazonHttpClient.java:1035)
          	at com.amazonaws.http.AmazonHttpClient$RequestExecutor.doExecute(AmazonHttpClient.java:747)
          	at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeWithTimer(AmazonHttpClient.java:721)
          	at com.amazonaws.http.AmazonHttpClient$RequestExecutor.execute(AmazonHttpClient.java:704)
          	at com.amazonaws.http.AmazonHttpClient$RequestExecutor.access$500(AmazonHttpClient.java:672)
          	at com.amazonaws.http.AmazonHttpClient$RequestExecutionBuilderImpl.execute(AmazonHttpClient.java:654)
          	at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:518)
          	at com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:4185)
          	at com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:4132)
          	at com.amazonaws.services.s3.AmazonS3Client.getObjectMetadata(AmazonS3Client.java:1245)
          	at org.apache.hadoop.fs.s3a.S3AFileSystem.getObjectMetadata(S3AFileSystem.java:1081)
          	at org.apache.hadoop.fs.s3a.S3AFileSystem.s3GetFileStatus(S3AFileSystem.java:1817)
          	... 13 more
          2017-03-24 16:56:37,470 INFO [CommitterEvent Processor #1] org.apache.hadoop.fs.s3a.commit.
          

          What's happening? resumably, the mock S3 client isn't getting as far as the MockedStagingCommitter in the AM: a new process is starting and so the special committer pushed into the FileSystem.get() cache isn't being found.

          Fix? Presumably to revert to the trick as the original mock tests did: patch in a new fs.s3a.impl class into the job submission and use it.

          Not we do not want to do that for the real IT tests.

          Show
          stevel@apache.org Steve Loughran added a comment - IT Test testMapFileOutputCommitter failures with unique IDs enabled This is failing as the unique ID goes onto the path such as dest/part-m-0000/index-job-attempt_01 , part-m-000/data-job-attempt_01 . As a result, the files index and data aren't found in the test or the reducer. The naming needs to go into the part-m-0000-$UUID/index and part-m-0000-$UUID/index . I've disabled unique filenames by default; adapted files to match for the partitioned committer, it doesn't work with mappers anyway, because of how things are handled in the commit (as far as I can tell, though I could be confused here.). Disabled the relevant IT test. TestStatingMRJob Failures Reasons for failure in Yetus unclear. Locally, appears to work, but in the AM log there's a stack warning that s3a isn't bonding 2017-03-24 16:56:37,468 WARN [CommitterEvent Processor #1] org.apache.hadoop.fs.s3a.S3AUtils: Failed to delete s3a: //bucket-name/output/path/_temporary java.nio.file.AccessDeniedException: s3a: //bucket-name/output/path/_temporary: getFileStatus on s3a://bucket-name/output/path/_temporary: com.amazonaws.services.s3.model.AmazonS3Exception: Forbidden (Service: Amazon S3; Status Code: 403; Error Code: 403 Forbidden; Request ID: 5F7696A22A66FAAF), S3 Extended Request ID: cVVOEgwybZL0YQbQ0fwFbPMv7GO/TOBAVJed//EQsQUX9dGNkgzCrLy1U6ZbMHmb9qQyKzZSOcw= at org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:159) at org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:102) at org.apache.hadoop.fs.s3a.S3AFileSystem.s3GetFileStatus(S3AFileSystem.java:1832) at org.apache.hadoop.fs.s3a.S3AFileSystem.innerGetFileStatus(S3AFileSystem.java:1802) at org.apache.hadoop.fs.s3a.S3AFileSystem.delete(S3AFileSystem.java:1416) at org.apache.hadoop.fs.s3a.S3AUtils.deleteWithWarning(S3AUtils.java:756) at org.apache.hadoop.fs.s3a.commit.staging.StagingS3GuardCommitter.deleteDestinationPaths(StagingS3GuardCommitter.java:817) at org.apache.hadoop.fs.s3a.commit.staging.StagingS3GuardCommitter.cleanup(StagingS3GuardCommitter.java:780) at org.apache.hadoop.fs.s3a.commit.staging.StagingS3GuardCommitter.commitJobInternal(StagingS3GuardCommitter.java:685) at org.apache.hadoop.fs.s3a.commit.staging.StagingS3GuardCommitter.commitJob(StagingS3GuardCommitter.java:625) at org.apache.hadoop.fs.s3a.commit.staging.MockedStagingCommitter.commitJob(MockedStagingCommitter.java:84) at org.apache.hadoop.mapreduce.v2.app.commit.CommitterEventHandler$EventProcessor.handleJobCommit(CommitterEventHandler.java:286) at org.apache.hadoop.mapreduce.v2.app.commit.CommitterEventHandler$EventProcessor.run(CommitterEventHandler.java:238) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) at java.lang. Thread .run( Thread .java:745) Caused by: com.amazonaws.services.s3.model.AmazonS3Exception: Forbidden (Service: Amazon S3; Status Code: 403; Error Code: 403 Forbidden; Request ID: 5F7696A22A66FAAF), S3 Extended Request ID: cVVOEgwybZL0YQbQ0fwFbPMv7GO/TOBAVJed //EQsQUX9dGNkgzCrLy1U6ZbMHmb9qQyKzZSOcw= at com.amazonaws.http.AmazonHttpClient$RequestExecutor.handleErrorResponse(AmazonHttpClient.java:1586) at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeOneRequest(AmazonHttpClient.java:1254) at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeHelper(AmazonHttpClient.java:1035) at com.amazonaws.http.AmazonHttpClient$RequestExecutor.doExecute(AmazonHttpClient.java:747) at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeWithTimer(AmazonHttpClient.java:721) at com.amazonaws.http.AmazonHttpClient$RequestExecutor.execute(AmazonHttpClient.java:704) at com.amazonaws.http.AmazonHttpClient$RequestExecutor.access$500(AmazonHttpClient.java:672) at com.amazonaws.http.AmazonHttpClient$RequestExecutionBuilderImpl.execute(AmazonHttpClient.java:654) at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:518) at com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:4185) at com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:4132) at com.amazonaws.services.s3.AmazonS3Client.getObjectMetadata(AmazonS3Client.java:1245) at org.apache.hadoop.fs.s3a.S3AFileSystem.getObjectMetadata(S3AFileSystem.java:1081) at org.apache.hadoop.fs.s3a.S3AFileSystem.s3GetFileStatus(S3AFileSystem.java:1817) ... 13 more 2017-03-24 16:56:37,470 INFO [CommitterEvent Processor #1] org.apache.hadoop.fs.s3a.commit. What's happening? resumably, the mock S3 client isn't getting as far as the MockedStagingCommitter in the AM: a new process is starting and so the special committer pushed into the FileSystem.get() cache isn't being found. Fix? Presumably to revert to the trick as the original mock tests did: patch in a new fs.s3a.impl class into the job submission and use it. Not we do not want to do that for the real IT tests.
          Hide
          stevel@apache.org Steve Loughran added a comment -

          Patch 022
          making FileCommitActions the way the s3 operations are accessed; the staging committers no longer have their own S3Client. Instead they pass the FileCommitActions instance down to the S3Util (I've also renamed it StagingUtil to make clear its about staging work, rather than generic s3 work). Fixing up the mock tests to handle this change was more than the production side changes.

          Testing:

          • serialized tests work.
          • parallel tests, found/fixed bug where pending commits to other bits of the filesystem were being found and aborted (getting prefix setup wrong on listMultipartUploads).
          • parallel committer tests are failing as they are all using the same temp dir under target/ for their local staging dir, and as job/task IDs are also shared, running >1 committer test in parallel will fail. I need to set up a per-fork temp dir.
          Show
          stevel@apache.org Steve Loughran added a comment - Patch 022 making FileCommitActions the way the s3 operations are accessed; the staging committers no longer have their own S3Client. Instead they pass the FileCommitActions instance down to the S3Util (I've also renamed it StagingUtil to make clear its about staging work, rather than generic s3 work). Fixing up the mock tests to handle this change was more than the production side changes. Testing: serialized tests work. parallel tests, found/fixed bug where pending commits to other bits of the filesystem were being found and aborted (getting prefix setup wrong on listMultipartUploads). parallel committer tests are failing as they are all using the same temp dir under target/ for their local staging dir, and as job/task IDs are also shared, running >1 committer test in parallel will fail . I need to set up a per-fork temp dir.
          Hide
          stevel@apache.org Steve Loughran added a comment -

          Patch 23

          Work done while mostly offline/travelling, so more housekeeping than featur. People on conference wifi get upset if you do s3 scale tests.

          • Unified constants across committer and with S3 core (e.g. same partition sized used for staging committer part uploads as block output)
          • made all the import structures consistent
          • WiP on documenting config options.
          • Magic committer's abort outcomes more explicit by moving to enum of outcomes rather than a simple success/fail, adding ABORTED and ABORT_FAILED to achieve this.

          Test changes primarily related to intermittent test failures; one problem was that if >1 committer test ran in parallel, they could interfere. The Fork ID is now used for the Job ID. That is trickier than you'd think, hence changes to the hadoop-aws POM.

          Next todo items

          • Rebase onto latest HADOOP-13345 ( after that catches up with trunk again)
          • Move staging onto s3a FS methods
          • instrument committer operations (maybe add counters for the committers? Or just have them access some new FS stats, "pending commits", "completed"...
          Show
          stevel@apache.org Steve Loughran added a comment - Patch 23 Work done while mostly offline/travelling, so more housekeeping than featur. People on conference wifi get upset if you do s3 scale tests. Unified constants across committer and with S3 core (e.g. same partition sized used for staging committer part uploads as block output) made all the import structures consistent WiP on documenting config options. Magic committer's abort outcomes more explicit by moving to enum of outcomes rather than a simple success/fail, adding ABORTED and ABORT_FAILED to achieve this. Test changes primarily related to intermittent test failures; one problem was that if >1 committer test ran in parallel, they could interfere. The Fork ID is now used for the Job ID. That is trickier than you'd think, hence changes to the hadoop-aws POM. Next todo items Rebase onto latest HADOOP-13345 ( after that catches up with trunk again) Move staging onto s3a FS methods instrument committer operations (maybe add counters for the committers? Or just have them access some new FS stats, "pending commits", "completed"...
          Hide
          stevel@apache.org Steve Loughran added a comment -

          Looks like the latest code is failing in TestStagingMRJob when there's no network as it fails to bond to the test object store. Even if there's binding to the mock FS, looks like something is also trying to talk to the real one

          "Finalizer" #3 daemon prio=8 os_prio=31 tid=0x00007fca50035000 nid=0x3803 in Object.wait() [0x0000700005281000]
             java.lang.Thread.State: WAITING (on object monitor)
          	at java.lang.Object.wait(Native Method)
          	- waiting on <0x00000007800339c8> (a java.lang.ref.ReferenceQueue$Lock)
          	at java.lang.ref.ReferenceQueue.remove(ReferenceQueue.java:143)
          	- locked <0x00000007800339c8> (a java.lang.ref.ReferenceQueue$Lock)
          	at java.lang.ref.ReferenceQueue.remove(ReferenceQueue.java:164)
          	at java.lang.ref.Finalizer$FinalizerThread.run(Finalizer.java:209)
          
          "Reference Handler" #2 daemon prio=10 os_prio=31 tid=0x00007fca4e80f000 nid=0x3603 in Object.wait() [0x000070000517e000]
             java.lang.Thread.State: WAITING (on object monitor)
          	at java.lang.Object.wait(Native Method)
          	- waiting on <0x0000000780033b80> (a java.lang.ref.Reference$Lock)
          	at java.lang.Object.wait(Object.java:502)
          	at java.lang.ref.Reference.tryHandlePending(Reference.java:191)
          	- locked <0x0000000780033b80> (a java.lang.ref.Reference$Lock)
          	at java.lang.ref.Reference$ReferenceHandler.run(Reference.java:153)
          
          "main" #1 prio=5 os_prio=31 tid=0x00007fca4d002000 nid=0x1e03 waiting on condition [0x000070000475e000]
             java.lang.Thread.State: TIMED_WAITING (sleeping)
          	at java.lang.Thread.sleep(Native Method)
          	at com.amazonaws.http.AmazonHttpClient$RequestExecutor.doPauseBeforeRetry(AmazonHttpClient.java:1654)
          	at com.amazonaws.http.AmazonHttpClient$RequestExecutor.pauseBeforeRetry(AmazonHttpClient.java:1628)
          	at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeOneRequest(AmazonHttpClient.java:1139)
          	at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeHelper(AmazonHttpClient.java:1035)
          	at com.amazonaws.http.AmazonHttpClient$RequestExecutor.doExecute(AmazonHttpClient.java:747)
          	at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeWithTimer(AmazonHttpClient.java:721)
          	at com.amazonaws.http.AmazonHttpClient$RequestExecutor.execute(AmazonHttpClient.java:704)
          	at com.amazonaws.http.AmazonHttpClient$RequestExecutor.access$500(AmazonHttpClient.java:672)
          	at com.amazonaws.http.AmazonHttpClient$RequestExecutionBuilderImpl.execute(AmazonHttpClient.java:654)
          	at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:518)
          	at com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:4185)
          	at com.amazonaws.services.s3.AmazonS3Client.getBucketRegionViaHeadRequest(AmazonS3Client.java:4903)
          	at com.amazonaws.services.s3.AmazonS3Client.fetchRegionFromCache(AmazonS3Client.java:4878)
          	at com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:4170)
          	at com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:4132)
          	at com.amazonaws.services.s3.AmazonS3Client.headBucket(AmazonS3Client.java:1302)
          	at com.amazonaws.services.s3.AmazonS3Client.doesBucketExist(AmazonS3Client.java:1259)
          	at org.apache.hadoop.fs.s3a.S3AFileSystem.verifyBucketExists(S3AFileSystem.java:317)
          	at org.apache.hadoop.fs.s3a.S3AFileSystem.initialize(S3AFileSystem.java:255)
          	at org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:3257)
          	at org.apache.hadoop.fs.FileSystem.access$200(FileSystem.java:123)
          	at org.apache.hadoop.fs.FileSystem$Cache.getInternal(FileSystem.java:3306)
          	at org.apache.hadoop.fs.FileSystem$Cache.get(FileSystem.java:3274)
          	at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:476)
          	at org.apache.hadoop.fs.Path.getFileSystem(Path.java:361)
          	at org.apache.hadoop.fs.s3a.commit.staging.MockedStagingCommitter.getDestination(MockedStagingCommitter.java:69)
          	at org.apache.hadoop.fs.s3a.commit.AbstractS3GuardCommitter.initOutput(AbstractS3GuardCommitter.java:122)
          	at org.apache.hadoop.fs.s3a.commit.AbstractS3GuardCommitter.<init>(AbstractS3GuardCommitter.java:85)
          	at org.apache.hadoop.fs.s3a.commit.AbstractS3GuardCommitter.<init>(AbstractS3GuardCommitter.java:114)
          	at org.apache.hadoop.fs.s3a.commit.staging.StagingS3GuardCommitter.<init>(StagingS3GuardCommitter.java:153)
          	at org.apache.hadoop.fs.s3a.commit.staging.MockedStagingCommitter.<init>(MockedStagingCommitter.java:49)
          	at org.apache.hadoop.fs.s3a.commit.staging.TestStagingMRJob$S3TextOutputFormat.getOutputCommitter(TestStagingMRJob.java:94)
          	- locked <0x0000000780082b60> (a org.apache.hadoop.fs.s3a.commit.staging.TestStagingMRJob$S3TextOutputFormat)
          	at org.apache.hadoop.mapreduce.v2.app.MRAppMaster$3.call(MRAppMaster.java:553)
          	at org.apache.hadoop.mapreduce.v2.app.MRAppMaster$3.call(MRAppMaster.java:535)
          	at org.apache.hadoop.mapreduce.v2.app.MRAppMaster.callWithJobClassLoader(MRAppMaster.java:1763)
          	at org.apache.hadoop.mapreduce.v2.app.MRAppMaster.createOutputCommitter(MRAppMaster.java:535)
          	at org.apache.hadoop.mapreduce.v2.app.MRAppMaster.serviceInit(MRAppMaster.java:312)
          	at org.apache.hadoop.service.AbstractService.init(AbstractService.java:163)
          	- locked <0x0000000780082cb0> (a java.lang.Object)
          	at org.apache.hadoop.mapreduce.v2.app.MRAppMaster$6.run(MRAppMaster.java:1721)
          	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:1965)
          	at org.apache.hadoop.mapreduce.v2.app.MRAppMaster.initAndStartAppMaster(MRAppMaster.java:1718)
          	at org.apache.hadoop.mapreduce.v2.app.MRAppMaster.main(MRAppMaster.java:1649)
          
          Show
          stevel@apache.org Steve Loughran added a comment - Looks like the latest code is failing in TestStagingMRJob when there's no network as it fails to bond to the test object store. Even if there's binding to the mock FS, looks like something is also trying to talk to the real one "Finalizer" #3 daemon prio=8 os_prio=31 tid=0x00007fca50035000 nid=0x3803 in Object .wait() [0x0000700005281000] java.lang. Thread .State: WAITING (on object monitor) at java.lang. Object .wait(Native Method) - waiting on <0x00000007800339c8> (a java.lang.ref.ReferenceQueue$Lock) at java.lang.ref.ReferenceQueue.remove(ReferenceQueue.java:143) - locked <0x00000007800339c8> (a java.lang.ref.ReferenceQueue$Lock) at java.lang.ref.ReferenceQueue.remove(ReferenceQueue.java:164) at java.lang.ref.Finalizer$FinalizerThread.run(Finalizer.java:209) "Reference Handler" #2 daemon prio=10 os_prio=31 tid=0x00007fca4e80f000 nid=0x3603 in Object .wait() [0x000070000517e000] java.lang. Thread .State: WAITING (on object monitor) at java.lang. Object .wait(Native Method) - waiting on <0x0000000780033b80> (a java.lang.ref.Reference$Lock) at java.lang. Object .wait( Object .java:502) at java.lang.ref.Reference.tryHandlePending(Reference.java:191) - locked <0x0000000780033b80> (a java.lang.ref.Reference$Lock) at java.lang.ref.Reference$ReferenceHandler.run(Reference.java:153) "main" #1 prio=5 os_prio=31 tid=0x00007fca4d002000 nid=0x1e03 waiting on condition [0x000070000475e000] java.lang. Thread .State: TIMED_WAITING (sleeping) at java.lang. Thread .sleep(Native Method) at com.amazonaws.http.AmazonHttpClient$RequestExecutor.doPauseBeforeRetry(AmazonHttpClient.java:1654) at com.amazonaws.http.AmazonHttpClient$RequestExecutor.pauseBeforeRetry(AmazonHttpClient.java:1628) at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeOneRequest(AmazonHttpClient.java:1139) at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeHelper(AmazonHttpClient.java:1035) at com.amazonaws.http.AmazonHttpClient$RequestExecutor.doExecute(AmazonHttpClient.java:747) at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeWithTimer(AmazonHttpClient.java:721) at com.amazonaws.http.AmazonHttpClient$RequestExecutor.execute(AmazonHttpClient.java:704) at com.amazonaws.http.AmazonHttpClient$RequestExecutor.access$500(AmazonHttpClient.java:672) at com.amazonaws.http.AmazonHttpClient$RequestExecutionBuilderImpl.execute(AmazonHttpClient.java:654) at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:518) at com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:4185) at com.amazonaws.services.s3.AmazonS3Client.getBucketRegionViaHeadRequest(AmazonS3Client.java:4903) at com.amazonaws.services.s3.AmazonS3Client.fetchRegionFromCache(AmazonS3Client.java:4878) at com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:4170) at com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:4132) at com.amazonaws.services.s3.AmazonS3Client.headBucket(AmazonS3Client.java:1302) at com.amazonaws.services.s3.AmazonS3Client.doesBucketExist(AmazonS3Client.java:1259) at org.apache.hadoop.fs.s3a.S3AFileSystem.verifyBucketExists(S3AFileSystem.java:317) at org.apache.hadoop.fs.s3a.S3AFileSystem.initialize(S3AFileSystem.java:255) at org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:3257) at org.apache.hadoop.fs.FileSystem.access$200(FileSystem.java:123) at org.apache.hadoop.fs.FileSystem$Cache.getInternal(FileSystem.java:3306) at org.apache.hadoop.fs.FileSystem$Cache.get(FileSystem.java:3274) at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:476) at org.apache.hadoop.fs.Path.getFileSystem(Path.java:361) at org.apache.hadoop.fs.s3a.commit.staging.MockedStagingCommitter.getDestination(MockedStagingCommitter.java:69) at org.apache.hadoop.fs.s3a.commit.AbstractS3GuardCommitter.initOutput(AbstractS3GuardCommitter.java:122) at org.apache.hadoop.fs.s3a.commit.AbstractS3GuardCommitter.<init>(AbstractS3GuardCommitter.java:85) at org.apache.hadoop.fs.s3a.commit.AbstractS3GuardCommitter.<init>(AbstractS3GuardCommitter.java:114) at org.apache.hadoop.fs.s3a.commit.staging.StagingS3GuardCommitter.<init>(StagingS3GuardCommitter.java:153) at org.apache.hadoop.fs.s3a.commit.staging.MockedStagingCommitter.<init>(MockedStagingCommitter.java:49) at org.apache.hadoop.fs.s3a.commit.staging.TestStagingMRJob$S3TextOutputFormat.getOutputCommitter(TestStagingMRJob.java:94) - locked <0x0000000780082b60> (a org.apache.hadoop.fs.s3a.commit.staging.TestStagingMRJob$S3TextOutputFormat) at org.apache.hadoop.mapreduce.v2.app.MRAppMaster$3.call(MRAppMaster.java:553) at org.apache.hadoop.mapreduce.v2.app.MRAppMaster$3.call(MRAppMaster.java:535) at org.apache.hadoop.mapreduce.v2.app.MRAppMaster.callWithJobClassLoader(MRAppMaster.java:1763) at org.apache.hadoop.mapreduce.v2.app.MRAppMaster.createOutputCommitter(MRAppMaster.java:535) at org.apache.hadoop.mapreduce.v2.app.MRAppMaster.serviceInit(MRAppMaster.java:312) at org.apache.hadoop.service.AbstractService.init(AbstractService.java:163) - locked <0x0000000780082cb0> (a java.lang. Object ) at org.apache.hadoop.mapreduce.v2.app.MRAppMaster$6.run(MRAppMaster.java:1721) 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:1965) at org.apache.hadoop.mapreduce.v2.app.MRAppMaster.initAndStartAppMaster(MRAppMaster.java:1718) at org.apache.hadoop.mapreduce.v2.app.MRAppMaster.main(MRAppMaster.java:1649)
          Hide
          stevel@apache.org Steve Loughran added a comment -

          +need to verify that initiate MPU and abort operations have retry logic in S3A; doesn't look like it's in the AWS SDK: we've already added this for the final POST.

          Show
          stevel@apache.org Steve Loughran added a comment - +need to verify that initiate MPU and abort operations have retry logic in S3A; doesn't look like it's in the AWS SDK: we've already added this for the final POST.
          Hide
          stevel@apache.org Steve Loughran added a comment -

          Patch 024

          The staging committers have all moved over to working with S3A via the S3A FS; write operation helper methods have been added/altered as appropriate. This pulls them in to the instrumentation & metrics, sets things up for s3guard (TODO). The hardest part of the migration was the mock tests, as they've been very brittle to changes inside. I've pulled TestMRStagingJob altogether, as its copy and upgrade to an IT test obsoletes it. The others are now working again. and all the helper classes cleaned up as they don't have an s3 client to pass around. Tests need to know that client errors/results are set in the FS, not specific committers.

          I need to think about what to do with mock directories in the commit process. I Don't want to have the code generating needless fake empty dirs, for the perf hit, or generating needless delete requests. I think I'd like

          1. the initial commit process to create the dest dir as a mock empty dir. This stops race conditions between >1 committer to same test
          2. individual file commits to do nothing about the parent dir
          3. end of job commit to explicitly delete the parent dirs if present, if the commit succeeded.
            I'll just add another action in WriteOperationsHelper here.
          Show
          stevel@apache.org Steve Loughran added a comment - Patch 024 The staging committers have all moved over to working with S3A via the S3A FS; write operation helper methods have been added/altered as appropriate. This pulls them in to the instrumentation & metrics, sets things up for s3guard (TODO). The hardest part of the migration was the mock tests, as they've been very brittle to changes inside. I've pulled TestMRStagingJob altogether, as its copy and upgrade to an IT test obsoletes it. The others are now working again. and all the helper classes cleaned up as they don't have an s3 client to pass around. Tests need to know that client errors/results are set in the FS, not specific committers. I need to think about what to do with mock directories in the commit process. I Don't want to have the code generating needless fake empty dirs, for the perf hit, or generating needless delete requests. I think I'd like the initial commit process to create the dest dir as a mock empty dir. This stops race conditions between >1 committer to same test individual file commits to do nothing about the parent dir end of job commit to explicitly delete the parent dirs if present, if the commit succeeded. I'll just add another action in WriteOperationsHelper here.
          Hide
          stevel@apache.org Steve Loughran added a comment -

          revert commit logic should consider what to do if the revert operation fails. Swallow? Retry? If the commit operation itself does retries, then the revert logic is only needed on network or auth problems, which means that it's likely to continue through the revert process.

          Maybe: add a counter in S3A FS for revert failures, extend the mock TestStagingCommitter test to include failures there to see what happens.

          Show
          stevel@apache.org Steve Loughran added a comment - revert commit logic should consider what to do if the revert operation fails. Swallow? Retry? If the commit operation itself does retries, then the revert logic is only needed on network or auth problems, which means that it's likely to continue through the revert process. Maybe: add a counter in S3A FS for revert failures, extend the mock TestStagingCommitter test to include failures there to see what happens.
          Hide
          fabbri Aaron Fabbri added a comment -

          Hi Steve Loughran. I applied the patch (with minor rebasing) and ran ITestS3A* integration tests.

          Tests run: 374, Failures: 7, Errors: 24, Skipped: 12

          What can I do to help on this? Is it ready to run Spark tests?

          I'm happy to do some coding or code reviewing.. (the former can help with understanding for the latter).. Just shout. I'm looking through the patch but feel free to focus my efforts.

          Show
          fabbri Aaron Fabbri added a comment - Hi Steve Loughran . I applied the patch (with minor rebasing) and ran ITestS3A* integration tests. Tests run: 374, Failures: 7, Errors: 24, Skipped: 12 What can I do to help on this? Is it ready to run Spark tests? I'm happy to do some coding or code reviewing.. (the former can help with understanding for the latter).. Just shout. I'm looking through the patch but feel free to focus my efforts.
          Hide
          stevel@apache.org Steve Loughran added a comment -

          ooh, thanks for playing with this. Will contact you direct for a discussion, how to hook up tests. Note that I've not actually wired this up to S3guard, and as its doing some low level PUT/POST calls bypass the FS API, things will get ouf of sync.

          I have spark tests downstream; They work for all but Parquet. Nothing insurmountable, org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat mandates a ParquetOutputCommitter or commutter, It just needs to accept to OutputCommittert when you don't want save summary files, which you should be doing against S3 for performance reasons alone.

          I'll show you how to get them up and running, though classpath issues will make life hard for you right now.

          Review-wise, start by running ITestStagingCommitProtocol#testCommitLifecycle() to get a walk through of the commit lifecycle

          Show
          stevel@apache.org Steve Loughran added a comment - ooh, thanks for playing with this. Will contact you direct for a discussion, how to hook up tests. Note that I've not actually wired this up to S3guard, and as its doing some low level PUT/POST calls bypass the FS API, things will get ouf of sync. I have spark tests downstream; They work for all but Parquet. Nothing insurmountable, org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat mandates a ParquetOutputCommitter or commutter, It just needs to accept to OutputCommittert when you don't want save summary files, which you should be doing against S3 for performance reasons alone. I'll show you how to get them up and running, though classpath issues will make life hard for you right now. Review-wise, start by running ITestStagingCommitProtocol#testCommitLifecycle() to get a walk through of the commit lifecycle
          Hide
          stevel@apache.org Steve Loughran added a comment -

          uploading: TLA+ spec of S3's core behaviour, though it doesn't model the eventual consistency.

          Show
          stevel@apache.org Steve Loughran added a comment - uploading: TLA+ spec of S3's core behaviour, though it doesn't model the eventual consistency.
          Hide
          stevel@apache.org Steve Loughran added a comment -

          Patch 026;

          • rebased to HADOOP-13345
          • the _SUCCESS marker file is now a JSON manifest file listing: committer, host, timestamp, files committed

          The success file lets us verify that the new committer was actually used (guess what: it wasn't), and we can add extra assertions on it. This is the toString value from a test run

          SuccessData{
            committer='PartitionedStagingCommitter',
            hostname='devbox.local',
            description='Task committer attempt_1493832493956_0001_m_000000_0',
            date='Wed May 03 18:28:41 BST 2017',
            filenames=[test/testMRJob/part-m-00000, test/testMRJob/part-m-00002, test/testMRJob/part-m-00001]
          }
          

          We could actually use this to collect low-level timing info on operations, retry counts, etc, even where the execution engine doesn't: something which could be used for diagnostics, perhaps. Checking the listed filenames with the actual set of files in the directory would be a way of verifying that the work is all visible.

          Show
          stevel@apache.org Steve Loughran added a comment - Patch 026; rebased to HADOOP-13345 the _SUCCESS marker file is now a JSON manifest file listing: committer, host, timestamp, files committed The success file lets us verify that the new committer was actually used (guess what: it wasn't), and we can add extra assertions on it. This is the toString value from a test run SuccessData{ committer='PartitionedStagingCommitter', hostname='devbox.local', description='Task committer attempt_1493832493956_0001_m_000000_0', date='Wed May 03 18:28:41 BST 2017', filenames=[test/testMRJob/part-m-00000, test/testMRJob/part-m-00002, test/testMRJob/part-m-00001] } We could actually use this to collect low-level timing info on operations, retry counts, etc, even where the execution engine doesn't: something which could be used for diagnostics, perhaps. Checking the listed filenames with the actual set of files in the directory would be a way of verifying that the work is all visible.
          Hide
          fabbri Aaron Fabbri added a comment -

          Thanks for the update Steve Loughran. I'll build the latest and play with it.

          Curious on the TLA+ spec, did you find the process of writing it useful?

          Show
          fabbri Aaron Fabbri added a comment - Thanks for the update Steve Loughran . I'll build the latest and play with it. Curious on the TLA+ spec, did you find the process of writing it useful?
          Hide
          stevel@apache.org Steve Loughran added a comment -

          View it as my TLA+ homework. What I'd really like is to have the Hadoop FS spec in a form which I could then use in other TLA+ specs. That is: with a model of a filesystem, can we show that the FileOutputCommitter commit algorithm works, at least in the context of "if rename is atomic". As then if you can show how a blobstore mocks the FS, then you can see where it falls down.

          Show
          stevel@apache.org Steve Loughran added a comment - View it as my TLA+ homework. What I'd really like is to have the Hadoop FS spec in a form which I could then use in other TLA+ specs. That is: with a model of a filesystem, can we show that the FileOutputCommitter commit algorithm works, at least in the context of "if rename is atomic". As then if you can show how a blobstore mocks the FS, then you can see where it falls down.
          Hide
          fabbri Aaron Fabbri added a comment -

          Been spending some time reviewing the patch. It is pretty large, so I haven't covered 100%, but here are some comments so far:

          + * This method was copied from
          + * {@code org.apache.hadoop.registry.client.binding.JsonSerDeser}.
          + * @param <T> Type to marshal.
          + */
          

          Can we move the dependency to a common place instead of copying?

          /**
          + * A factory for committers implementing the {@link PathOutputCommitter}
          + * methods, and so can be used from {@link FileOutputFormat}.
          + * The base implementation returns {@link FileOutputFormat} instances.
          + */
          +public class PathOutputCommitterFactory extends Configured {
          

          In the comment, did you mean "FileOutputCommitter instances"?

                     bytes = putObject();
                   }
                 } else {
          -        // there has already been at least one block scheduled for upload;
          -        // put up the current then wait
          -        if (hasBlock && block.hasData()) {
          +        // there's an MPU in progress';
          +        // IF there is more data to upload, or no data has yet been uploaded,
          +        // PUT the final block
          +        if (hasBlock &&
          +            (block.hasData() || multiPartUpload.getPartsSubmitted() == 0)) {
          

          This to allow for zero byte files, right? Looks like you had to make similar changes elsewhere. Did you cover this new behavior with a test case?

          +  public void abortOutstandingMultipartUploads(long seconds)
          +      throws IOException {
          +    Preconditions.checkArgument(seconds >=0);
          +    Date purgeBefore =
          +        new Date(new Date().getTime() - seconds * 1000);
          +    LOG.debug("Purging outstanding multipart uploads older than {}",
          +        purgeBefore);
          +    try {
          +      transfers.abortMultipartUploads(bucket, purgeBefore);
          +    } catch (AmazonServiceException e) {
          +      throw translateException("purging multipart uploads", bucket, e);
          +    }
          

          Thinking of how multipart purge interacts with magic committer.

          Currently someone can config fs.s3a.multipart.purge.age to 0. I'm wondering if
          we shouldn't increase the minimum or log a warning when magic committer is also
          enabled? Maybe it at least deserves some special treatment in the docs?

              * Returns the S3 client used by this filesystem.
              * @return AmazonS3Client
              */
          -  AmazonS3 getAmazonS3Client() {
          +  public AmazonS3 getAmazonS3Client() {
               return s3;
             }
          

          Not @VisibleForTesting? Looks like you just want to subclass/override. Can we make it protected instead?

              * @param key input key
              * @return the fully qualified path including URI scheme and bucket name.
              */
          -  Path keyToQualifiedPath(String key) {
          +  public Path keyToQualifiedPath(String key) {
          

          Should we annotate the interface stability or audience here?

                * @return the upload result containing the ID
                * @throws IOException IO problem
                */
          -    String initiateMultiPartUpload() throws IOException {
          +    public String initiateMultiPartUpload() throws IOException {
                 LOG.debug("Initiating Multipart upload");
                 final InitiateMultipartUploadRequest initiateMPURequest =
                     new InitiateMultipartUploadRequest(bucket,
          

          Audience annotation?

          +    /**
          +     * Finalize a multipart PUT operation.
          +     * This completes the upload, and, if that works, calls
          +     * {@link #finishedWrite(String, long)} to update the filesystem.
          +     * @param uploadId multipart operation Id
          +     * @param partETags list of partial uploads
          +     * @return the result of the operation.
          +     * @throws IOException on problems.
          +     */
          

          Semi-related: I've been thinking of splitting up S3AFileSystem in the
          future. Seems like we could separate WriteOperationHelper, and maybe put
          other S3-level operations (i.e. below anything that cares about MetadataStore)
          into a separate class.

          +    public int abortMultipartUploadsUnderPath(String prefix)
          +        throws IOException {
          +      int count = 0;
          +      for (MultipartUpload upload : listMultipartUploads(prefix)) {
          +        try {
          +          abortMultipartCommit(upload);
          +          count++;
          +        } catch (FileNotFoundException e) {
          +          LOG.debug("Already aborted: {}", upload.getKey(), e);
          

          I wonder if we'll need to deal with retryable exceptions here, instead of
          propagating.

          +     * @return the upload initiated
          +     * @throws IOException on problems
          +     */
          +    public PutObjectResult putObjectAndFinalize(
          +        PutObjectRequest putObjectRequest,
          +        int length) throws IOException {
          +      PutObjectResult result = putObject(putObjectRequest);
          +      finishedWrite(putObjectRequest.getKey(), length);
          

          Also wondering about error handling / retries here. I don't know if there
          is a difference between direct put and TransferManager in terms of built-in
          retry logic.

          +    /**
          +     * Revert a commit by deleting the file.
          +     * TODO: Policy regarding creating a mock empty parent directory.
          +     * @param destKey destination key
          +     * @throws IOException due to inability to delete a directory or file.
          +     */
          +    public void revertCommit(String destKey) throws IOException {
          

          Does the directory need to exist after revert? Can we just say it doesn't?

          +  /**
          +   * Delete a path quietly: failures are logged at DEBUG.
          +   * @param fs filesystem
          +   * @param path path
          +   * @param recursive recursive?
          +   */
          +  public static void deleteWithWarning(FileSystem fs,
          

          /quietly: failures are logged at DEBUG/: failures are logged at WARN/

          +public abstract class Abstract3GuardCommitterFactory
          

          "AbstractS3GuardCommitterFactory" (Missing the 'S')

          +  public static List<String> splitPathToElements(Path path) {
          +    String uriPath = path.toUri().getPath();
          +    checkArgument(!uriPath.isEmpty(), "empty path");
          +    checkArgument(uriPath.charAt(0) == '/', "path is relative");
          +    if ("/".equals(uriPath)) {
          +      // special case: empty list
          +      return new ArrayList<>(0);
          +    }
          +    path.depth();
          

          Meant to delete this last line?

          +  /**
          +   * Verify that that path is a delayed commit path.
          

          nit: /that/the/ or /that//

          +
          +/**
          + * Adds the code needed for S3A integration.
          + * It's pulled out to keep S3A FS class slightly less complex.
          

          Thanks for the separation here.

          + * Configuration options:
          + * <pre>
          + *   : temporary local FS directory
          + *   : intermediate directory on a cluster-wide FS (can be HDFS or a consistent
          + *   s3 endpoint).
          

          Couldn't parse this comment. Is there supposed to be a config name before the ':'?

          +  public void testConcurrentCommitTaskWithSubDir() throws Exception {
          +    Job job = newJob();
          +    FileOutputFormat.setOutputPath(job, outDir);
          +    final Configuration conf = job.getConfiguration();
          +/*
          +
          +    conf.setClass("fs.file.impl", RLFS.class, FileSystem.class);
          +    FileSystem.closeAll();
          +*/
          

          There are a couple of spots with commented-out code and TODOs still. Note to clean these up.

          +    // if this fails with "directory is already locked" set umask to 0022
          +    cluster = new MiniDFSCluster(conf, 1, true, null);
          +    //cluster = new MiniDFSCluster.Builder(new Configuration()).build();
          

          ditto.

          +/*
          +  @Override
          +  protected void deleteObject(String key) throws InvalidRequestException {
          +    // force into sharing the existing mock entry point
          +    getAmazonS3Client().deleteObject(new DeleteObjectRequest(getBucket(), key));
          +  }
          +*/
          

          same.

          +
          +    @Override
          +    public void setFaults(Faults... faults) {
          +      injection.setFaults(faults);
          

          I only saw this used once in AbstractITCommitProtocol which sets a
          commitJob failure. Curious to hear your plans on more fault injection. Hope we can discuss in a call soon.

          Finally, a reminder that your log4j.properties diff will need to be commented out or removed eventually. The settings here are useful for testing and dev though.

          Show
          fabbri Aaron Fabbri added a comment - Been spending some time reviewing the patch. It is pretty large, so I haven't covered 100%, but here are some comments so far: + * This method was copied from + * {@code org.apache.hadoop.registry.client.binding.JsonSerDeser}. + * @param <T> Type to marshal. + */ Can we move the dependency to a common place instead of copying? /** + * A factory for committers implementing the {@link PathOutputCommitter} + * methods, and so can be used from {@link FileOutputFormat}. + * The base implementation returns {@link FileOutputFormat} instances. + */ + public class PathOutputCommitterFactory extends Configured { In the comment, did you mean "FileOutputCommitter instances"? bytes = putObject(); } } else { - // there has already been at least one block scheduled for upload; - // put up the current then wait - if (hasBlock && block.hasData()) { + // there's an MPU in progress'; + // IF there is more data to upload, or no data has yet been uploaded, + // PUT the final block + if (hasBlock && + (block.hasData() || multiPartUpload.getPartsSubmitted() == 0)) { This to allow for zero byte files, right? Looks like you had to make similar changes elsewhere. Did you cover this new behavior with a test case? + public void abortOutstandingMultipartUploads( long seconds) + throws IOException { + Preconditions.checkArgument(seconds >=0); + Date purgeBefore = + new Date( new Date().getTime() - seconds * 1000); + LOG.debug( "Purging outstanding multipart uploads older than {}" , + purgeBefore); + try { + transfers.abortMultipartUploads(bucket, purgeBefore); + } catch (AmazonServiceException e) { + throw translateException( "purging multipart uploads" , bucket, e); + } Thinking of how multipart purge interacts with magic committer. Currently someone can config fs.s3a.multipart.purge.age to 0. I'm wondering if we shouldn't increase the minimum or log a warning when magic committer is also enabled? Maybe it at least deserves some special treatment in the docs? * Returns the S3 client used by this filesystem. * @ return AmazonS3Client */ - AmazonS3 getAmazonS3Client() { + public AmazonS3 getAmazonS3Client() { return s3; } Not @VisibleForTesting? Looks like you just want to subclass/override. Can we make it protected instead? * @param key input key * @ return the fully qualified path including URI scheme and bucket name. */ - Path keyToQualifiedPath( String key) { + public Path keyToQualifiedPath( String key) { Should we annotate the interface stability or audience here? * @ return the upload result containing the ID * @ throws IOException IO problem */ - String initiateMultiPartUpload() throws IOException { + public String initiateMultiPartUpload() throws IOException { LOG.debug( "Initiating Multipart upload" ); final InitiateMultipartUploadRequest initiateMPURequest = new InitiateMultipartUploadRequest(bucket, Audience annotation? + /** + * Finalize a multipart PUT operation. + * This completes the upload, and, if that works, calls + * {@link #finishedWrite( String , long )} to update the filesystem. + * @param uploadId multipart operation Id + * @param partETags list of partial uploads + * @ return the result of the operation. + * @ throws IOException on problems. + */ Semi-related: I've been thinking of splitting up S3AFileSystem in the future. Seems like we could separate WriteOperationHelper, and maybe put other S3-level operations (i.e. below anything that cares about MetadataStore) into a separate class. + public int abortMultipartUploadsUnderPath( String prefix) + throws IOException { + int count = 0; + for (MultipartUpload upload : listMultipartUploads(prefix)) { + try { + abortMultipartCommit(upload); + count++; + } catch (FileNotFoundException e) { + LOG.debug( "Already aborted: {}" , upload.getKey(), e); I wonder if we'll need to deal with retryable exceptions here, instead of propagating. + * @ return the upload initiated + * @ throws IOException on problems + */ + public PutObjectResult putObjectAndFinalize( + PutObjectRequest putObjectRequest, + int length) throws IOException { + PutObjectResult result = putObject(putObjectRequest); + finishedWrite(putObjectRequest.getKey(), length); Also wondering about error handling / retries here. I don't know if there is a difference between direct put and TransferManager in terms of built-in retry logic. + /** + * Revert a commit by deleting the file. + * TODO: Policy regarding creating a mock empty parent directory. + * @param destKey destination key + * @ throws IOException due to inability to delete a directory or file. + */ + public void revertCommit( String destKey) throws IOException { Does the directory need to exist after revert? Can we just say it doesn't? + /** + * Delete a path quietly: failures are logged at DEBUG. + * @param fs filesystem + * @param path path + * @param recursive recursive? + */ + public static void deleteWithWarning(FileSystem fs, /quietly: failures are logged at DEBUG/: failures are logged at WARN/ + public abstract class Abstract3GuardCommitterFactory "AbstractS3GuardCommitterFactory" (Missing the 'S') + public static List< String > splitPathToElements(Path path) { + String uriPath = path.toUri().getPath(); + checkArgument(!uriPath.isEmpty(), "empty path" ); + checkArgument(uriPath.charAt(0) == '/', "path is relative" ); + if ( "/" .equals(uriPath)) { + // special case : empty list + return new ArrayList<>(0); + } + path.depth(); Meant to delete this last line? + /** + * Verify that that path is a delayed commit path. nit: /that/the/ or /that// + +/** + * Adds the code needed for S3A integration. + * It's pulled out to keep S3A FS class slightly less complex. Thanks for the separation here. + * Configuration options: + * <pre> + * : temporary local FS directory + * : intermediate directory on a cluster-wide FS (can be HDFS or a consistent + * s3 endpoint). Couldn't parse this comment. Is there supposed to be a config name before the ':'? + public void testConcurrentCommitTaskWithSubDir() throws Exception { + Job job = newJob(); + FileOutputFormat.setOutputPath(job, outDir); + final Configuration conf = job.getConfiguration(); +/* + + conf.setClass( "fs.file.impl" , RLFS.class, FileSystem.class); + FileSystem.closeAll(); +*/ There are a couple of spots with commented-out code and TODOs still. Note to clean these up. + // if this fails with "directory is already locked" set umask to 0022 + cluster = new MiniDFSCluster(conf, 1, true , null ); + //cluster = new MiniDFSCluster.Builder( new Configuration()).build(); ditto. +/* + @Override + protected void deleteObject( String key) throws InvalidRequestException { + // force into sharing the existing mock entry point + getAmazonS3Client().deleteObject( new DeleteObjectRequest(getBucket(), key)); + } +*/ same. + + @Override + public void setFaults(Faults... faults) { + injection.setFaults(faults); I only saw this used once in AbstractITCommitProtocol which sets a commitJob failure. Curious to hear your plans on more fault injection. Hope we can discuss in a call soon. Finally, a reminder that your log4j.properties diff will need to be commented out or removed eventually. The settings here are useful for testing and dev though.
          Hide
          stevel@apache.org Steve Loughran added a comment -

          son ser/deser

          good point: done. The registry one is now a subclass, with special handling of the load phase where it can scan for a marker string before trying to parse the JSON. That's needed there to fail fast if the entry at the ZK node isn't the expected JSON type, or indeed, not JSON at all.

          In the comment, did you mean "FileOutputCommitter instances"?

          yes. fixed

          0 byte files

          in ITestS3ACommitOperations .testCreateAbortEmptyFile

          Currently someone can config fs.s3a.multipart.purge.age to 0.

          yeah. To be honest, its pretty dangerous to instantiate an fs client with this set to a low value on any shared bucket. because of the risk of killing ongoing writes...it's why we've backed off from doing thsi in tests altogether.

          the commit protocol merely increases the window of risk, no matter which committer is used (as for both committers, it holds from task commit onwards, so stays pending while all other commits take place)

          +1 for docs.

          public getAmazonS3Client()

          that was exported for the original netflix import; now I've finished moving to the WriteOperationHelper methods it can be package scoped again. Done

          Should we annotate the interface stability or audience here?

          The whole class is tagged private/evolving; javadocs say "If cast to

          {@code S3AFileSystem}

          , extra methods and features may be accessed. Consider those private and unstable."

          Same for the other scope queries. We've tagged the API as unstable. What's important though is to limit the damage which someone using the APIs can do. package-scoping getAmazonS3Client() matters here because it's free access to the internals, bypassing s3guard & our metrics and monitoring. The others seem less critical.

          Making WriteOperationHelper standalone

          I'd originally created it to isolate the ops needed for the block output stream: with the constructor private it simply wasn't possible for outside callers to get at the WriteOperations. The committer expands the ops to cover everything needed for committing, 20+ methods, and an external interface. So it probably is time to do it, with a package scoped constructor and uprating private S3A methods it was calling to package scoped too.

          We should also go through all exported S3AFS methods and restrict access just to stay in control.

          I can do all of these in this patch, which will increase patch size, but avoid me facing lots of merge/rebase pain

          retry logic

          yes, ignoring it right now. But we should have retries on all the ops. I think we should start with a core design for retrying in our existing code, then reuse it here. Especially if we do something clever with lambda expressions. On that topic. I'm thinking of something like:

          MPU = execute(new RetryAllButAuth(), () -> initiateMultipartUpload())
          

          Added some more detail on that to HADOOP-14303. Something like that should be good for testability too.

          I think that could go into the s3guard branch, where I'd pick it up and use here, or we merge this in and then adopt.

          revert

          Does the directory need to exist after revert? Can we just say it doesn't?

          OK

          deleteWithWarning

          Fixed javadocs, aslo changed catch to IOEs only. Did the same for deleteQuietly().

          typo in Abstract3GuardCommitterFactory

          This is why reviews are so good. I'd missed that completely.

          "Configuration options:" javadoc

          cut this out; we can use MD files for covering configs

          TODOs.

          yes, they came in with the original code. Big ones: some about tests, plus we need to consider whether the the temp dir in HDFS should be configurable to something other than /tmp/$USER. Also I commented out some other bits.

          Anyway, thx for the review, will submit a patch with them in by the end of the weke.

          meanwhile

          • spark master now has the module needed for testing downstream
          • One big outstanding issue which doesn't have todo is actually having this work propery with s3guard. So far the code just lives alongside, but I need to make sure that the commit operations make things visible in s3guard, and that all the magic stuff gets created with the consistency we need
          Show
          stevel@apache.org Steve Loughran added a comment - son ser/deser good point: done. The registry one is now a subclass, with special handling of the load phase where it can scan for a marker string before trying to parse the JSON. That's needed there to fail fast if the entry at the ZK node isn't the expected JSON type, or indeed, not JSON at all. In the comment, did you mean "FileOutputCommitter instances"? yes. fixed 0 byte files in ITestS3ACommitOperations .testCreateAbortEmptyFile Currently someone can config fs.s3a.multipart.purge.age to 0. yeah. To be honest, its pretty dangerous to instantiate an fs client with this set to a low value on any shared bucket. because of the risk of killing ongoing writes...it's why we've backed off from doing thsi in tests altogether. the commit protocol merely increases the window of risk, no matter which committer is used (as for both committers, it holds from task commit onwards, so stays pending while all other commits take place) +1 for docs. public getAmazonS3Client() that was exported for the original netflix import; now I've finished moving to the WriteOperationHelper methods it can be package scoped again. Done Should we annotate the interface stability or audience here? The whole class is tagged private/evolving; javadocs say "If cast to {@code S3AFileSystem} , extra methods and features may be accessed. Consider those private and unstable." Same for the other scope queries. We've tagged the API as unstable. What's important though is to limit the damage which someone using the APIs can do. package-scoping getAmazonS3Client() matters here because it's free access to the internals, bypassing s3guard & our metrics and monitoring. The others seem less critical. Making WriteOperationHelper standalone I'd originally created it to isolate the ops needed for the block output stream: with the constructor private it simply wasn't possible for outside callers to get at the WriteOperations. The committer expands the ops to cover everything needed for committing, 20+ methods, and an external interface. So it probably is time to do it, with a package scoped constructor and uprating private S3A methods it was calling to package scoped too. We should also go through all exported S3AFS methods and restrict access just to stay in control. I can do all of these in this patch, which will increase patch size, but avoid me facing lots of merge/rebase pain retry logic yes, ignoring it right now. But we should have retries on all the ops. I think we should start with a core design for retrying in our existing code, then reuse it here. Especially if we do something clever with lambda expressions. On that topic. I'm thinking of something like: MPU = execute( new RetryAllButAuth(), () -> initiateMultipartUpload()) Added some more detail on that to HADOOP-14303 . Something like that should be good for testability too. I think that could go into the s3guard branch, where I'd pick it up and use here, or we merge this in and then adopt. revert Does the directory need to exist after revert? Can we just say it doesn't? OK deleteWithWarning Fixed javadocs, aslo changed catch to IOEs only. Did the same for deleteQuietly() . typo in Abstract3GuardCommitterFactory This is why reviews are so good. I'd missed that completely. "Configuration options:" javadoc cut this out; we can use MD files for covering configs TODOs. yes, they came in with the original code. Big ones: some about tests, plus we need to consider whether the the temp dir in HDFS should be configurable to something other than /tmp/$USER. Also I commented out some other bits. Anyway, thx for the review, will submit a patch with them in by the end of the weke. meanwhile spark master now has the module needed for testing downstream One big outstanding issue which doesn't have todo is actually having this work propery with s3guard. So far the code just lives alongside, but I need to make sure that the commit operations make things visible in s3guard, and that all the magic stuff gets created with the consistency we need
          Hide
          stevel@apache.org Steve Loughran added a comment -

          Patch 026

          Key changes:

          1. tried to address Aarons comments;
          2. implemented Magic Committer with MR test to validate.

          review changes

          mostly commented on above. I've pulled WriteOperationsHelper out, and added a new class AwsCall which takes a closure and executes, it translating exceptions

            <T> T execute(String action, String path, Operation<T> operation)
                throws IOException {
              try {
                return operation.execute();
              } catch (AmazonClientException e) {
                throw S3AUtils.translateException(action, path, e);
              }
            }
          

          This is where we can add retry logic, throttling, etc. I've not done that, just lined things up for it to go in across the module. Example of use

          calls.execute("upload part",
                  request.getKey(),
                  () -> owner.uploadPart(request));
          

          Magic Committer

          works, at least as far as the IT Tests go. Done by pulling up all the staging code from the base class and switching to that logic for execution of operations

          Show
          stevel@apache.org Steve Loughran added a comment - Patch 026 Key changes: tried to address Aarons comments; implemented Magic Committer with MR test to validate. review changes mostly commented on above. I've pulled WriteOperationsHelper out, and added a new class AwsCall which takes a closure and executes, it translating exceptions <T> T execute( String action, String path, Operation<T> operation) throws IOException { try { return operation.execute(); } catch (AmazonClientException e) { throw S3AUtils.translateException(action, path, e); } } This is where we can add retry logic, throttling, etc. I've not done that, just lined things up for it to go in across the module. Example of use calls.execute( "upload part" , request.getKey(), () -> owner.uploadPart(request)); Magic Committer works, at least as far as the IT Tests go. Done by pulling up all the staging code from the base class and switching to that logic for execution of operations
          Hide
          stevel@apache.org Steve Loughran added a comment -

          Patch 027

          1. fix javadoc, checkstyle complaints, especially by encapsulating fields in the json-serialized data.
          2. strip out stuff which is superfluous now the Magic committer is using the staging committer commit code
          3. tests for the uploadFile(File, dest....) routine used by the staging committer; now handles 0 byte files & has a leaner API

          This is now starting to feel like it is coming together. I'm still focusing on the staging committer(s) as the ones to be ready to use first, but the fact that things are unified and code cut is a sign of tightening up.

          TODO

          • the bits in the code with TODO in them.
          • isolated tests for the Task class
          • instrumentation
          • scale tests.

          I'm looking at doing the scale tests in spark code, downstream of spark itself. Spark now pulls in the hadoop-aws JARs (& azure) if built right, so out-the-git-repo spark/master is ready to be used in the tests. Doing it in spark will (a) make it easier for me to write queries running in the test JVM itself, which offers: parallelism, easier to inject faults, and to get some log output when it all goes wrong.

          Testing: s3 ireland. I'm doing a test with s3guard local right now, and things are failing there, so somehow I've broken that. Moral: we should start asking for test runs against s3guard as well as "classic"

          Show
          stevel@apache.org Steve Loughran added a comment - Patch 027 fix javadoc, checkstyle complaints, especially by encapsulating fields in the json-serialized data. strip out stuff which is superfluous now the Magic committer is using the staging committer commit code tests for the uploadFile(File, dest....) routine used by the staging committer; now handles 0 byte files & has a leaner API This is now starting to feel like it is coming together. I'm still focusing on the staging committer(s) as the ones to be ready to use first, but the fact that things are unified and code cut is a sign of tightening up. TODO the bits in the code with TODO in them. isolated tests for the Task class instrumentation scale tests. I'm looking at doing the scale tests in spark code, downstream of spark itself. Spark now pulls in the hadoop-aws JARs (& azure) if built right, so out-the-git-repo spark/master is ready to be used in the tests. Doing it in spark will (a) make it easier for me to write queries running in the test JVM itself, which offers: parallelism, easier to inject faults, and to get some log output when it all goes wrong. Testing: s3 ireland. I'm doing a test with s3guard local right now, and things are failing there, so somehow I've broken that. Moral: we should start asking for test runs against s3guard as well as "classic"
          Hide
          stevel@apache.org Steve Loughran added a comment -

          HADOOP-13786 Patch 028

          • fixed up the javdoc/checkstyle complaints
          • metrics added, logged in MR test
          • improved cleanup of dirs, tests for this
          • use transfer manager for uploading the success data. I haven't done this for the actual block output stream, but I can, either in this patch or later
          • everything tested with s3guard & localb, as well as without it

          feature wise, things are stablising. I'm taking this opportunity to do the big refactoring work to committers &c

          I'm not doing any retries on operations. I think we must do that on the final commit of the put, the way S3ABlockOutputStream does. Why? A transient failure there loses the whole commit, hence job.

          test-wise: reaching limit of what we can do without fault injection; next step is downstream stuff. My spark cloudexamples can do this, indeed, it will read in the success data and verify the committer used is the one invoked, so verifying that the committer is being used.

          Show
          stevel@apache.org Steve Loughran added a comment - HADOOP-13786 Patch 028 fixed up the javdoc/checkstyle complaints metrics added, logged in MR test improved cleanup of dirs, tests for this use transfer manager for uploading the success data. I haven't done this for the actual block output stream, but I can, either in this patch or later everything tested with s3guard & localb, as well as without it feature wise, things are stablising. I'm taking this opportunity to do the big refactoring work to committers &c I'm not doing any retries on operations. I think we must do that on the final commit of the put, the way S3ABlockOutputStream does. Why? A transient failure there loses the whole commit, hence job. test-wise: reaching limit of what we can do without fault injection; next step is downstream stuff. My spark cloudexamples can do this, indeed, it will read in the success data and verify the committer used is the one invoked, so verifying that the committer is being used.
          Hide
          stevel@apache.org Steve Loughran added a comment -

          Patch 029

          1. name of MultiplePendingUploads structure is #PendingSet# in classname and docs
          2. MR job IT now does 20 files on a -Dscale test run, for a little bit more scaleup
          3. renaming "delayed commit" -> "pending commit" in code", still "magic commit" in docs.
          4. Rename TestS3AHugePendingCommits so scale tests run in the serial part of a test run
          5. metrics: track job+ task commit/fail, propagate to tests via the _SUCCESS data where they are now logged
          6. review/update the docs, going from design to actual. Maybe I should get a PDF of the previous version & attach it here for readable history
          7. address some yetus complaints, mostly about non-encapsulated fields in datastructures used in tests, datastructures there because java doesn't do tuples. I've done some and am going to stop at the rest, as checkstyle is behind the times.

          As well as reviews, having people play with this will be nice.

          I'm going to focus on the docs, make sure the issues are covered and structure right, then work on the spark tests in
          https://github.com/hortonworks-spark/cloud-integration . the _SUCCESS data structure can be read there even without requiring to be built against a compatible Hadoop version (it's just JSON to read...) and so tests are already verifying which committer is being used. I plan to do more there, and then create some tests designed to go through the bits of Spark which use the more complex extensions of the protocol (`CREATE TABLE` for example).

          Show
          stevel@apache.org Steve Loughran added a comment - Patch 029 name of MultiplePendingUploads structure is #PendingSet# in classname and docs MR job IT now does 20 files on a -Dscale test run, for a little bit more scaleup renaming "delayed commit" -> "pending commit" in code", still "magic commit" in docs. Rename TestS3AHugePendingCommits so scale tests run in the serial part of a test run metrics: track job+ task commit/fail, propagate to tests via the _SUCCESS data where they are now logged review/update the docs, going from design to actual. Maybe I should get a PDF of the previous version & attach it here for readable history address some yetus complaints, mostly about non-encapsulated fields in datastructures used in tests, datastructures there because java doesn't do tuples. I've done some and am going to stop at the rest, as checkstyle is behind the times. As well as reviews, having people play with this will be nice. I'm going to focus on the docs, make sure the issues are covered and structure right, then work on the spark tests in https://github.com/hortonworks-spark/cloud-integration . the _SUCCESS data structure can be read there even without requiring to be built against a compatible Hadoop version (it's just JSON to read...) and so tests are already verifying which committer is being used. I plan to do more there, and then create some tests designed to go through the bits of Spark which use the more complex extensions of the protocol (`CREATE TABLE` for example).
          Hide
          stevel@apache.org Steve Loughran added a comment -

          Got a stack trace downstream of this from some invalid thread pool values, no actual cue as to which was invalid. Adding some meaningful validation to the next path

          2017-05-18 21:22:37,385 [ScalaTest-main] INFO  cloud.CloudSuite (CloudLogging.scala:logInfo(56)) - Using committer unset
          *** RUN ABORTED ***
            java.lang.RuntimeException: Unable to load a Suite class that was discovered in the runpath: com.hortonworks.spark.cloud.s3.S3ANumbersSuite
            at org.scalatest.tools.DiscoverySuite$.getSuiteInstance(DiscoverySuite.scala:84)
            at org.scalatest.tools.DiscoverySuite$$anonfun$1.apply(DiscoverySuite.scala:38)
            at org.scalatest.tools.DiscoverySuite$$anonfun$1.apply(DiscoverySuite.scala:37)
            at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
            at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
            at scala.collection.Iterator$class.foreach(Iterator.scala:893)
            at scala.collection.AbstractIterator.foreach(Iterator.scala:1336)
            at scala.collection.IterableLike$class.foreach(IterableLike.scala:72)
            at scala.collection.AbstractIterable.foreach(Iterable.scala:54)
            at scala.collection.TraversableLike$class.map(TraversableLike.scala:234)
            ...
            Cause: java.lang.IllegalArgumentException:
            at java.util.concurrent.ThreadPoolExecutor.<init>(ThreadPoolExecutor.java:1307)
            at java.util.concurrent.ThreadPoolExecutor.<init>(ThreadPoolExecutor.java:1230)
            at org.apache.hadoop.fs.s3a.S3AFileSystem.initialize(S3AFileSystem.java:280)
            at org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:3258)
            at org.apache.hadoop.fs.FileSystem.access$200(FileSystem.java:123)
            at org.apache.hadoop.fs.FileSystem$Cache.getInternal(FileSystem.java:3307)
            at org.apache.hadoop.fs.FileSystem$Cache.getUnique(FileSystem.java:3281)
            at org.apache.hadoop.fs.FileSystem.newInstance(FileSystem.java:529)
            at com.hortonworks.spark.cloud.CloudSuite.createFilesystem(CloudSuite.scala:166)
            at com.hortonworks.spark.cloud.s3.S3ANumbersSuite.com$hortonworks$spark$cloud$s3$S3ATestSetup$$super$createFilesystem(S3ANumbersSuite.scala:22)
          
          Show
          stevel@apache.org Steve Loughran added a comment - Got a stack trace downstream of this from some invalid thread pool values, no actual cue as to which was invalid. Adding some meaningful validation to the next path 2017-05-18 21:22:37,385 [ScalaTest-main] INFO cloud.CloudSuite (CloudLogging.scala:logInfo(56)) - Using committer unset *** RUN ABORTED *** java.lang.RuntimeException: Unable to load a Suite class that was discovered in the runpath: com.hortonworks.spark.cloud.s3.S3ANumbersSuite at org.scalatest.tools.DiscoverySuite$.getSuiteInstance(DiscoverySuite.scala:84) at org.scalatest.tools.DiscoverySuite$$anonfun$1.apply(DiscoverySuite.scala:38) at org.scalatest.tools.DiscoverySuite$$anonfun$1.apply(DiscoverySuite.scala:37) at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234) at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234) at scala.collection.Iterator$class.foreach(Iterator.scala:893) at scala.collection.AbstractIterator.foreach(Iterator.scala:1336) at scala.collection.IterableLike$class.foreach(IterableLike.scala:72) at scala.collection.AbstractIterable.foreach(Iterable.scala:54) at scala.collection.TraversableLike$class.map(TraversableLike.scala:234) ... Cause: java.lang.IllegalArgumentException: at java.util.concurrent.ThreadPoolExecutor.<init>(ThreadPoolExecutor.java:1307) at java.util.concurrent.ThreadPoolExecutor.<init>(ThreadPoolExecutor.java:1230) at org.apache.hadoop.fs.s3a.S3AFileSystem.initialize(S3AFileSystem.java:280) at org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:3258) at org.apache.hadoop.fs.FileSystem.access$200(FileSystem.java:123) at org.apache.hadoop.fs.FileSystem$Cache.getInternal(FileSystem.java:3307) at org.apache.hadoop.fs.FileSystem$Cache.getUnique(FileSystem.java:3281) at org.apache.hadoop.fs.FileSystem.newInstance(FileSystem.java:529) at com.hortonworks.spark.cloud.CloudSuite.createFilesystem(CloudSuite.scala:166) at com.hortonworks.spark.cloud.s3.S3ANumbersSuite.com$hortonworks$spark$cloud$s3$S3ATestSetup$$ super $createFilesystem(S3ANumbersSuite.scala:22)
          Hide
          stevel@apache.org Steve Loughran added a comment -

          Patch 030: evolution based on integration testing with the InconsistentAmazonS3Client enabled, s3guard on/off, in Spark, so using its workflow.

          • the _SUCCESS marker contains more information & diagnostics
          • various bits of tuning shown (making cleanup resilient to inconsistencies in list vs actual)
          • docs

          It's in sync with commit 0fbb4aa in https://github.com/hortonworks-spark/cloud-integration; as is the documentation

          The core integration tests are working; more is always welcome...I plan to scale things up & create 1+ test designed to work on large clusters. This is all just querying data, but it adds validation of the data from the _SUCCESS marker, which is new.

          Example printing of success marker data

          2017-06-02 19:59:19,780 [ScalaTest-main-running-S3ACommitDataframeSuite] INFO  s3.S3AOperations (Logging.scala:logInfo(54)) - success data at s3a://hwdev-steve-new/cloud-integration/DELAY_LISTING_ME/S3ACommitDataframeSuite/dataframe-committer/partitioned/orc/_SUCCESS : SuccessData{committer='PartitionedStagingCommitter', hostname='HW13176.cotham.uk', description='Task committer attempt_20170602195913_0000_m_000000_0', date='Fri Jun 02 19:59:17 BST 2017', filenames=[/cloud-integration/DELAY_LISTING_ME/S3ACommitDataframeSuite/dataframe-committer/partitioned/orc/part-00000-f22d488c-dad0-4fa5-8ca4-8d00b058c77c-c000.snappy.orc]}
          2017-06-02 19:59:19,781 [ScalaTest-main-running-S3ACommitDataframeSuite] INFO  s3.S3AOperations (Logging.scala:logInfo(54)) - Metrics:
            S3guard_metadatastore_put_path_latency50thPercentileLatency = 548156
            S3guard_metadatastore_put_path_latency75thPercentileLatency = 548156
            S3guard_metadatastore_put_path_latency90thPercentileLatency = 548156
            S3guard_metadatastore_put_path_latency95thPercentileLatency = 548156
            S3guard_metadatastore_put_path_latency99thPercentileLatency = 548156
            S3guard_metadatastore_put_path_latencyNumOps = 1
            committer_bytes_committed = 384
            committer_commits_aborted = 0
            committer_commits_completed = 1
            committer_commits_created = 1
            committer_commits_failed = 0
            committer_commits_reverted = 0
            committer_jobs_completed = 1
            committer_jobs_failed = 0
            committer_tasks_completed = 1
            committer_tasks_failed = 0
            directories_created = 1
            directories_deleted = 0
            fake_directories_deleted = 6
            files_copied = 0
            files_copied_bytes = 0
            files_created = 0
            files_deleted = 2
            ignored_errors = 1
            object_continue_list_requests = 0
            object_copy_requests = 0
            object_delete_requests = 2
            object_list_requests = 5
            object_metadata_requests = 8
            object_multipart_aborted = 0
            object_put_bytes = 384
            object_put_bytes_pending = 0
            object_put_requests = 2
            object_put_requests_active = 0
            object_put_requests_completed = 2
            op_copy_from_local_file = 0
            op_exists = 2
            op_get_file_status = 4
            op_glob_status = 0
            op_is_directory = 0
            op_is_file = 0
            op_list_files = 0
            op_list_located_status = 0
            op_list_status = 0
            op_mkdirs = 0
            op_rename = 0
            s3guard_metadatastore_initialization = 0
            s3guard_metadatastore_put_path_request = 2
            stream_aborted = 0
            stream_backward_seek_operations = 0
            stream_bytes_backwards_on_seek = 0
            stream_bytes_discarded_in_abort = 0
            stream_bytes_read = 0
            stream_bytes_read_in_close = 0
            stream_bytes_skipped_on_seek = 0
            stream_close_operations = 0
            stream_closed = 0
            stream_forward_seek_operations = 0
            stream_opened = 0
            stream_read_exceptions = 0
            stream_read_fully_operations = 0
            stream_read_operations = 0
            stream_read_operations_incomplete = 0
            stream_seek_operations = 0
            stream_write_block_uploads = 0
            stream_write_block_uploads_aborted = 0
            stream_write_block_uploads_active = 0
            stream_write_block_uploads_committed = 0
            stream_write_block_uploads_data_pending = 0
            stream_write_block_uploads_pending = 0
            stream_write_failures = 0
            stream_write_total_data = 0
            stream_write_total_time = 0
          
          2017-06-02 19:59:19,782 [ScalaTest-main-running-S3ACommitDataframeSuite] INFO  s3.S3AOperations (Logging.scala:logInfo(54)) - Diagnostics:
            fs.s3a.committer.magic.enabled = true
            fs.s3a.metadatastore.authoritative = false
            fs.s3a.metadatastore.impl = org.apache.hadoop.fs.s3a.s3guard.LocalMetadataStore
          
          Show
          stevel@apache.org Steve Loughran added a comment - Patch 030: evolution based on integration testing with the InconsistentAmazonS3Client enabled, s3guard on/off, in Spark, so using its workflow. the _SUCCESS marker contains more information & diagnostics various bits of tuning shown (making cleanup resilient to inconsistencies in list vs actual) docs It's in sync with commit 0fbb4aa in https://github.com/hortonworks-spark/cloud-integration ; as is the documentation The core integration tests are working; more is always welcome...I plan to scale things up & create 1+ test designed to work on large clusters. This is all just querying data, but it adds validation of the data from the _SUCCESS marker, which is new. Example printing of success marker data 2017-06-02 19:59:19,780 [ScalaTest-main-running-S3ACommitDataframeSuite] INFO s3.S3AOperations (Logging.scala:logInfo(54)) - success data at s3a: //hwdev-steve- new /cloud-integration/DELAY_LISTING_ME/S3ACommitDataframeSuite/dataframe-committer/partitioned/orc/_SUCCESS : SuccessData{committer='PartitionedStagingCommitter', hostname='HW13176.cotham.uk', description='Task committer attempt_20170602195913_0000_m_000000_0', date='Fri Jun 02 19:59:17 BST 2017', filenames=[/cloud-integration/DELAY_LISTING_ME/S3ACommitDataframeSuite/dataframe-committer/partitioned/orc/part-00000-f22d488c-dad0-4fa5-8ca4-8d00b058c77c-c000.snappy.orc]} 2017-06-02 19:59:19,781 [ScalaTest-main-running-S3ACommitDataframeSuite] INFO s3.S3AOperations (Logging.scala:logInfo(54)) - Metrics: S3guard_metadatastore_put_path_latency50thPercentileLatency = 548156 S3guard_metadatastore_put_path_latency75thPercentileLatency = 548156 S3guard_metadatastore_put_path_latency90thPercentileLatency = 548156 S3guard_metadatastore_put_path_latency95thPercentileLatency = 548156 S3guard_metadatastore_put_path_latency99thPercentileLatency = 548156 S3guard_metadatastore_put_path_latencyNumOps = 1 committer_bytes_committed = 384 committer_commits_aborted = 0 committer_commits_completed = 1 committer_commits_created = 1 committer_commits_failed = 0 committer_commits_reverted = 0 committer_jobs_completed = 1 committer_jobs_failed = 0 committer_tasks_completed = 1 committer_tasks_failed = 0 directories_created = 1 directories_deleted = 0 fake_directories_deleted = 6 files_copied = 0 files_copied_bytes = 0 files_created = 0 files_deleted = 2 ignored_errors = 1 object_continue_list_requests = 0 object_copy_requests = 0 object_delete_requests = 2 object_list_requests = 5 object_metadata_requests = 8 object_multipart_aborted = 0 object_put_bytes = 384 object_put_bytes_pending = 0 object_put_requests = 2 object_put_requests_active = 0 object_put_requests_completed = 2 op_copy_from_local_file = 0 op_exists = 2 op_get_file_status = 4 op_glob_status = 0 op_is_directory = 0 op_is_file = 0 op_list_files = 0 op_list_located_status = 0 op_list_status = 0 op_mkdirs = 0 op_rename = 0 s3guard_metadatastore_initialization = 0 s3guard_metadatastore_put_path_request = 2 stream_aborted = 0 stream_backward_seek_operations = 0 stream_bytes_backwards_on_seek = 0 stream_bytes_discarded_in_abort = 0 stream_bytes_read = 0 stream_bytes_read_in_close = 0 stream_bytes_skipped_on_seek = 0 stream_close_operations = 0 stream_closed = 0 stream_forward_seek_operations = 0 stream_opened = 0 stream_read_exceptions = 0 stream_read_fully_operations = 0 stream_read_operations = 0 stream_read_operations_incomplete = 0 stream_seek_operations = 0 stream_write_block_uploads = 0 stream_write_block_uploads_aborted = 0 stream_write_block_uploads_active = 0 stream_write_block_uploads_committed = 0 stream_write_block_uploads_data_pending = 0 stream_write_block_uploads_pending = 0 stream_write_failures = 0 stream_write_total_data = 0 stream_write_total_time = 0 2017-06-02 19:59:19,782 [ScalaTest-main-running-S3ACommitDataframeSuite] INFO s3.S3AOperations (Logging.scala:logInfo(54)) - Diagnostics: fs.s3a.committer.magic.enabled = true fs.s3a.metadatastore.authoritative = false fs.s3a.metadatastore.impl = org.apache.hadoop.fs.s3a.s3guard.LocalMetadataStore
          Hide
          fabbri Aaron Fabbri added a comment -

          Rebased v30 patch onto latest feature branch (shout if you want me to post the rebased patch). Ran ITestS3ACommit* successfully. Reading through the latest patch and playing with the code some.

          qq: Should we have some test assumptions that s3guard is enabled for ITestS3ACommitActions, or is it useful to test w/o S3Guard?

          Steve Loughran there is a lot to go over here. I'd love any small tasks you can assign to me to get my hands dirty working on the code.

          Show
          fabbri Aaron Fabbri added a comment - Rebased v30 patch onto latest feature branch (shout if you want me to post the rebased patch). Ran ITestS3ACommit* successfully. Reading through the latest patch and playing with the code some. qq: Should we have some test assumptions that s3guard is enabled for ITestS3ACommitActions, or is it useful to test w/o S3Guard? Steve Loughran there is a lot to go over here. I'd love any small tasks you can assign to me to get my hands dirty working on the code.
          Hide
          stevel@apache.org Steve Loughran added a comment -

          Patch 031

          • in sync with latest HADOOP-13345 branch
          • testing commit logic with the inconsistent s3a client wherever possible; helps validate the algorithms work. Test probes have changed in places to handle this, primarily by adding some sleeps when inconsistency is enabled.
          Show
          stevel@apache.org Steve Loughran added a comment - Patch 031 in sync with latest HADOOP-13345 branch testing commit logic with the inconsistent s3a client wherever possible; helps validate the algorithms work. Test probes have changed in places to handle this, primarily by adding some sleeps when inconsistency is enabled.
          Hide
          stevel@apache.org Steve Loughran added a comment -

          Patch 032

          Main change here is implementing failure resilience on all the commit algorithms. This is done with some major (and long outstanding) enhancments to S3A's ability to recognise different AWS exceptions, including throttling, and implement retries depending on what's happened

          Specifically

          1. new Exceptions for specific problems (throttling, bad request)
          2. S3AUtils.translateException to map errors to them
          3. S3ARetryPolicy to choose actions base on exception type. Failfast for unrecoverables (network, auth), fail with exponential backoff for throttling, fail with limited retries for everything else.
          4. extended S3a Lambda integration to wrap any l-exp with the retry policy.
          5. WriteOperationsHelper to wrap all operations with retry invocation example

          Essentially, it's now trivial to take any S3 interaction and wrap it with retry logic. This works for all the existing FS ops too

            protected FileStatus getFileStatus(Path path) throws IOException {
              return lambda.retry("getFileStatus", path.toString(), true,
                  () -> fs.getFileStatus(path));
            }
          

          I've only fitted this to the writeOperationsHelper new methods & the commit logic; left the retry logic commiting an{{S3ABlockOutputStream}} alone, and doing nothing to the S3AFS methods instead. But we should be doing the latter for all operations which don't go through AWS xfer manager

          I'd like to pull this feature into HADOOP-13345 once I've had a bit more experience of failure tuning (example: does making BadRequest retriable help?). It can be the basis for all the DDB retry logic too. Then we can look at wrapping all other FS operations which are at risk of failing.

          Show
          stevel@apache.org Steve Loughran added a comment - Patch 032 Main change here is implementing failure resilience on all the commit algorithms. This is done with some major (and long outstanding) enhancments to S3A's ability to recognise different AWS exceptions, including throttling, and implement retries depending on what's happened Specifically new Exceptions for specific problems ( throttling , bad request ) S3AUtils.translateException to map errors to them S3ARetryPolicy to choose actions base on exception type. Failfast for unrecoverables (network, auth), fail with exponential backoff for throttling, fail with limited retries for everything else. extended S3a Lambda integration to wrap any l-exp with the retry policy. WriteOperationsHelper to wrap all operations with retry invocation example Essentially, it's now trivial to take any S3 interaction and wrap it with retry logic. This works for all the existing FS ops too protected FileStatus getFileStatus(Path path) throws IOException { return lambda.retry( "getFileStatus" , path.toString(), true , () -> fs.getFileStatus(path)); } I've only fitted this to the writeOperationsHelper new methods & the commit logic; left the retry logic commiting an{{S3ABlockOutputStream}} alone, and doing nothing to the S3AFS methods instead. But we should be doing the latter for all operations which don't go through AWS xfer manager I'd like to pull this feature into HADOOP-13345 once I've had a bit more experience of failure tuning (example: does making BadRequest retriable help?). It can be the basis for all the DDB retry logic too. Then we can look at wrapping all other FS operations which are at risk of failing.
          Hide
          stevel@apache.org Steve Loughran added a comment -

          Patch 033

          • fault injection to include raising throttle exceptions
          • test all the commit operations
          • fix bugs in retry logic which surfaced, including existing one in S3ABlockOutputStream.
          • clean up WriteOperationHelper with move to S3A lambda
          • retry policy considers ConnectTimeoutException as retriable.

          All the new commit actions are tested as resilient to throttle exceptions, which is how I verify that they are being handled by the s3a lambda calls.

          What is not wrapped/handled is the underlying implementations of S3AFS (getFileStatus, create, open, ), all of which need to be. That can be done once this is merged into S3A.

          + (checkstyle fixed where appropriate)

          Testing, S3a Ireland. I'm getting intermittent BadRequest exceptions on ITestS3AEncryptionSSEC and TestS3AContractRootDir.testRecursiveRootListing, when run as part of the maven parallel test run but not when run alone. These are all part of the sequential test. I also got some in ITestS3ACommitOperations which implied the FS it was getting didn't have fault injection enabled; explicitly asking for a new FS fixed that. Something is up with JVM and test recycling, I fear

          Show
          stevel@apache.org Steve Loughran added a comment - Patch 033 fault injection to include raising throttle exceptions test all the commit operations fix bugs in retry logic which surfaced, including existing one in S3ABlockOutputStream. clean up WriteOperationHelper with move to S3A lambda retry policy considers ConnectTimeoutException as retriable. All the new commit actions are tested as resilient to throttle exceptions, which is how I verify that they are being handled by the s3a lambda calls. What is not wrapped/handled is the underlying implementations of S3AFS (getFileStatus, create, open, ), all of which need to be. That can be done once this is merged into S3A. + (checkstyle fixed where appropriate) Testing, S3a Ireland. I'm getting intermittent BadRequest exceptions on ITestS3AEncryptionSSEC and TestS3AContractRootDir.testRecursiveRootListing , when run as part of the maven parallel test run but not when run alone . These are all part of the sequential test. I also got some in ITestS3ACommitOperations which implied the FS it was getting didn't have fault injection enabled; explicitly asking for a new FS fixed that. Something is up with JVM and test recycling, I fear
          Hide
          ehiggs Ewan Higgs added a comment -

          Hi,
          In the latest patch (033), in InconsistentAmazonS3Client I see the following line uses inconsistent formatting styles (String.format and slf4j):

            @Override
            public String toString() {
              return String.format(
                  "Inconsistent S3 Client with"
                      + " %s msec delay, substring %s, delay probability %s;"
                      + " throttle probability %s"
                      + "; failure limit {}, failure count {}",
                  delayKeyMsec, delayKeySubstring, delayKeyProbability,
                  throttleProbability, failureLimit, failureCounter.get());
            }
          

          This won't work. The braces will be ignored by the formatter. e.g.:

          public class A {
              public static void main(String args[]) { 
                  int x = 0;
                  int y = 1;
                  String str = String.format("Here is a string mixing percent s: %s and bracey {}", x, y);
                  System.out.println(str);
              }
          }
          
          $ java A
          Here is a string mixing percent s: 0 and bracey {}
          
          Show
          ehiggs Ewan Higgs added a comment - Hi, In the latest patch (033), in InconsistentAmazonS3Client I see the following line uses inconsistent formatting styles ( String.format and slf4j ): @Override public String toString() { return String .format( "Inconsistent S3 Client with" + " %s msec delay, substring %s, delay probability %s;" + " throttle probability %s" + "; failure limit {}, failure count {}" , delayKeyMsec, delayKeySubstring, delayKeyProbability, throttleProbability, failureLimit, failureCounter.get()); } This won't work. The braces will be ignored by the formatter. e.g.: public class A { public static void main( String args[]) { int x = 0; int y = 1; String str = String .format( "Here is a string mixing percent s: %s and bracey {}" , x, y); System .out.println(str); } } $ java A Here is a string mixing percent s: 0 and bracey {}
          Hide
          stevel@apache.org Steve Loughran added a comment -

          Ewan -yeah, that's what findbugs told me off about. Fixed in my local copy https://github.com/steveloughran/hadoop/tree/s3guard/HADOOP-13786-committer, just not submitted as a new patch yet as there's not been enough change. If you are planning to review it (yes please!) grab that code or let me I know and I'll stick the latest version up. (not much else has changed, documentation of algorithms done)

          Show
          stevel@apache.org Steve Loughran added a comment - Ewan -yeah, that's what findbugs told me off about. Fixed in my local copy https://github.com/steveloughran/hadoop/tree/s3guard/HADOOP-13786-committer , just not submitted as a new patch yet as there's not been enough change. If you are planning to review it (yes please!) grab that code or let me I know and I'll stick the latest version up. (not much else has changed, documentation of algorithms done)
          Hide
          ehiggs Ewan Higgs added a comment -

          Steve Loughran, I took a look at your Github branch but it's interleaved with patches from different branches. This makes it hard to follow which parts are changing for HADOOP-13786. If you could, please upload an up to date patch and I'll resume reviewing. Thanks!

          Show
          ehiggs Ewan Higgs added a comment - Steve Loughran , I took a look at your Github branch but it's interleaved with patches from different branches. This makes it hard to follow which parts are changing for HADOOP-13786 . If you could, please upload an up to date patch and I'll resume reviewing. Thanks!
          Hide
          stevel@apache.org Steve Loughran added a comment -

          ewan, thanks, I'll do that. It tries to have all the committer patches rebased on top of s3guard; if you do a diff of between it and HADOOP-13345 the changes are smaller. But there are a lot of commits on top; time to squash them

          Show
          stevel@apache.org Steve Loughran added a comment - ewan, thanks, I'll do that. It tries to have all the committer patches rebased on top of s3guard; if you do a diff of between it and HADOOP-13345 the changes are smaller. But there are a lot of commits on top; time to squash them
          Hide
          stevel@apache.org Steve Loughran added a comment -

          stack trace surfaced in a test run in Spark & magic,

          Driver stacktrace:
            at org.apache.spark.scheduler.DAGScheduler.org$apache$spark$scheduler$DAGScheduler$$failJobAndIndependentStages(DAGScheduler.scala:1570)
            at org.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1.apply(DAGScheduler.scala:1558)
            at org.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1.apply(DAGScheduler.scala:1557)
            at scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59)
            at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:48)
            at org.apache.spark.scheduler.DAGScheduler.abortStage(DAGScheduler.scala:1557)
            at org.apache.spark.scheduler.DAGScheduler$$anonfun$handleTaskSetFailed$1.apply(DAGScheduler.scala:819)
            at org.apache.spark.scheduler.DAGScheduler$$anonfun$handleTaskSetFailed$1.apply(DAGScheduler.scala:819)
            at scala.Option.foreach(Option.scala:257)
            at org.apache.spark.scheduler.DAGScheduler.handleTaskSetFailed(DAGScheduler.scala:819)
            ...
            Cause: org.apache.spark.SparkException: Task failed while writing rows.
            at org.apache.spark.sql.execution.datasources.FileFormatWriter$.org$apache$spark$sql$execution$datasources$FileFormatWriter$$executeTask(FileFormatWriter.scala:264)
            at org.apache.spark.sql.execution.datasources.FileFormatWriter$$anonfun$write$1.apply(FileFormatWriter.scala:184)
            at org.apache.spark.sql.execution.datasources.FileFormatWriter$$anonfun$write$1.apply(FileFormatWriter.scala:183)
            at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:87)
            at org.apache.spark.scheduler.Task.run(Task.scala:108)
            at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:341)
            at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
            at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
            at java.lang.Thread.run(Thread.java:745)
            ...
            Cause: java.io.FileNotFoundException: No such file or directory: s3a://hwdev-steve-new/cloud-integration/DELAY_LISTING_ME/S3ACommitDataframeSuite/dataframe-committer/committer-magic-orc/orc/__magic/app-attempt-0000/tasks/attempt_20170712151213_0001_m_000000_0/__base/part-00000-cecbf507-be09-4db5-a853-798e9677b0b3-c000.snappy.orc
            at org.apache.hadoop.fs.s3a.S3AFileSystem.s3GetFileStatus(S3AFileSystem.java:2111)
            at org.apache.hadoop.fs.s3a.S3AFileSystem.innerGetFileStatus(S3AFileSystem.java:2006)
            at org.apache.hadoop.fs.s3a.S3AFileSystem.getFileStatus(S3AFileSystem.java:1949)
            at org.apache.spark.sql.execution.datasources.FileFormatWriter$ExecuteWriteTask$class.getFileSize(FileFormatWriter.scala:297)
            at org.apache.spark.sql.execution.datasources.FileFormatWriter$SingleDirectoryWriteTask.getFileSize(FileFormatWriter.scala:305)
            at org.apache.spark.sql.execution.datasources.FileFormatWriter$SingleDirectoryWriteTask.releaseResources(FileFormatWriter.scala:361)
            at org.apache.spark.sql.execution.datasources.FileFormatWriter$SingleDirectoryWriteTask.execute(FileFormatWriter.scala:347)
            at org.apache.spark.sql.execution.datasources.FileFormatWriter$$anonfun$org$apache$spark$sql$execution$datasources$FileFormatWriter$$executeTask$3.apply(FileFormatWriter.scala:248)
            at org.apache.spark.sql.execution.datasources.FileFormatWriter$$anonfun$org$apache$spark$sql$execution$datasources$FileFormatWriter$$executeTask$3.apply(FileFormatWriter.scala:246)
            at org.apache.spark.util.Utils$.tryWithSafeFinallyAndFailureCallbacks(Utils.scala:1385)
          

          Issue is that after writing data, spark tries to work out how big it is, and in the magic committer its not there

          numOutputBytes += getFileSize(taskAttemptContext.getConfiguration, currentPath)
          

          Options:

          1. spark catch FNFE and downgrade, it being just a small detail for status. But this does at least failfast if something went wrong with a file writer.
          2. Magic to PUT a 0-byte file under the destination path. Dangerous in different ways, but it will stop that breaking. This would need to be done at the end of the write
          3. Extra devious: recognise a 404 under a magic path, check to see if there is a .pending file of the same name, and return its length instead. That's more expensive, especially given the pending file would need to be loaded. If the summary length was actually attached as an attribute in the PUT, of course, it'd be visible

          Option 3 is the kind of complication which gets scary fast. In either case, there's trouble if something expects to load the file and its not there.

          Show
          stevel@apache.org Steve Loughran added a comment - stack trace surfaced in a test run in Spark & magic, Driver stacktrace: at org.apache.spark.scheduler.DAGScheduler.org$apache$spark$scheduler$DAGScheduler$$failJobAndIndependentStages(DAGScheduler.scala:1570) at org.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1.apply(DAGScheduler.scala:1558) at org.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1.apply(DAGScheduler.scala:1557) at scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59) at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:48) at org.apache.spark.scheduler.DAGScheduler.abortStage(DAGScheduler.scala:1557) at org.apache.spark.scheduler.DAGScheduler$$anonfun$handleTaskSetFailed$1.apply(DAGScheduler.scala:819) at org.apache.spark.scheduler.DAGScheduler$$anonfun$handleTaskSetFailed$1.apply(DAGScheduler.scala:819) at scala.Option.foreach(Option.scala:257) at org.apache.spark.scheduler.DAGScheduler.handleTaskSetFailed(DAGScheduler.scala:819) ... Cause: org.apache.spark.SparkException: Task failed while writing rows. at org.apache.spark.sql.execution.datasources.FileFormatWriter$.org$apache$spark$sql$execution$datasources$FileFormatWriter$$executeTask(FileFormatWriter.scala:264) at org.apache.spark.sql.execution.datasources.FileFormatWriter$$anonfun$write$1.apply(FileFormatWriter.scala:184) at org.apache.spark.sql.execution.datasources.FileFormatWriter$$anonfun$write$1.apply(FileFormatWriter.scala:183) at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:87) at org.apache.spark.scheduler.Task.run(Task.scala:108) at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:341) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) at java.lang. Thread .run( Thread .java:745) ... Cause: java.io.FileNotFoundException: No such file or directory: s3a: //hwdev-steve- new /cloud-integration/DELAY_LISTING_ME/S3ACommitDataframeSuite/dataframe-committer/committer-magic-orc/orc/__magic/app-attempt-0000/tasks/attempt_20170712151213_0001_m_000000_0/__base/part-00000-cecbf507-be09-4db5-a853-798e9677b0b3-c000.snappy.orc at org.apache.hadoop.fs.s3a.S3AFileSystem.s3GetFileStatus(S3AFileSystem.java:2111) at org.apache.hadoop.fs.s3a.S3AFileSystem.innerGetFileStatus(S3AFileSystem.java:2006) at org.apache.hadoop.fs.s3a.S3AFileSystem.getFileStatus(S3AFileSystem.java:1949) at org.apache.spark.sql.execution.datasources.FileFormatWriter$ExecuteWriteTask$class.getFileSize(FileFormatWriter.scala:297) at org.apache.spark.sql.execution.datasources.FileFormatWriter$SingleDirectoryWriteTask.getFileSize(FileFormatWriter.scala:305) at org.apache.spark.sql.execution.datasources.FileFormatWriter$SingleDirectoryWriteTask.releaseResources(FileFormatWriter.scala:361) at org.apache.spark.sql.execution.datasources.FileFormatWriter$SingleDirectoryWriteTask.execute(FileFormatWriter.scala:347) at org.apache.spark.sql.execution.datasources.FileFormatWriter$$anonfun$org$apache$spark$sql$execution$datasources$FileFormatWriter$$executeTask$3.apply(FileFormatWriter.scala:248) at org.apache.spark.sql.execution.datasources.FileFormatWriter$$anonfun$org$apache$spark$sql$execution$datasources$FileFormatWriter$$executeTask$3.apply(FileFormatWriter.scala:246) at org.apache.spark.util.Utils$.tryWithSafeFinallyAndFailureCallbacks(Utils.scala:1385) Issue is that after writing data, spark tries to work out how big it is, and in the magic committer its not there numOutputBytes += getFileSize(taskAttemptContext.getConfiguration, currentPath) Options: spark catch FNFE and downgrade, it being just a small detail for status. But this does at least failfast if something went wrong with a file writer. Magic to PUT a 0-byte file under the destination path. Dangerous in different ways, but it will stop that breaking. This would need to be done at the end of the write Extra devious: recognise a 404 under a magic path, check to see if there is a .pending file of the same name, and return its length instead. That's more expensive, especially given the pending file would need to be loaded. If the summary length was actually attached as an attribute in the PUT, of course, it'd be visible Option 3 is the kind of complication which gets scary fast. In either case, there's trouble if something expects to load the file and its not there.
          Hide
          stevel@apache.org Steve Loughran added a comment -

          Need to rethink the factory model so as to support different committers for different filesystems, e.g. one for wasb, s3, ozone.

          How about you can register a commiter per schema, something like

          mapred.committer.factory.s3a=o.a.fs.s3a.committer.DynamicS3ACommitterFactory
          mapred.committer.factory.wasb=o.a.fs.wasb=WasbCommitterFactory
          # and default
          mapred.committer.factory=o.a.mapred.library.FileOutputCommitterFactory
          

          We'd look at the schema of the dest dir, then for a matching factory, with the default being the fallback. IT's up to each FS schema's factory to decide what committer it returns; for s3a it'll be per-bucket policy, probably default to staging as it doesn't require anything special in the dest FS

          Show
          stevel@apache.org Steve Loughran added a comment - Need to rethink the factory model so as to support different committers for different filesystems, e.g. one for wasb, s3, ozone. How about you can register a commiter per schema, something like mapred.committer.factory.s3a=o.a.fs.s3a.committer.DynamicS3ACommitterFactory mapred.committer.factory.wasb=o.a.fs.wasb=WasbCommitterFactory # and default mapred.committer.factory=o.a.mapred.library.FileOutputCommitterFactory We'd look at the schema of the dest dir, then for a matching factory, with the default being the fallback. IT's up to each FS schema's factory to decide what committer it returns; for s3a it'll be per-bucket policy, probably default to staging as it doesn't require anything special in the dest FS
          Hide
          stevel@apache.org Steve Loughran added a comment -

          Patch 035

          • Merges in some changes from Ewan Higgs
          • per-schema factory so that different filesystems/object stores can declare their own committer, and not interfere with the choice of the others.
            example:

          Example:

          mapreduce.pathoutputcommitter.factory.scheme.s3a=org.apache.hadoop.fs.s3a.commit.DynamicCommitterFactory
          

          This switches s3a to using a committer factory whose choice of committer is then based on the value of fs.s3a.committer.name. All other filesystems get the default committer set in the option "mapreduce.pathoutputcommitter.factory.class"

          Show
          stevel@apache.org Steve Loughran added a comment - Patch 035 Merges in some changes from Ewan Higgs per-schema factory so that different filesystems/object stores can declare their own committer, and not interfere with the choice of the others. example: Example: mapreduce.pathoutputcommitter.factory.scheme.s3a=org.apache.hadoop.fs.s3a.commit.DynamicCommitterFactory This switches s3a to using a committer factory whose choice of committer is then based on the value of fs.s3a.committer.name . All other filesystems get the default committer set in the option "mapreduce.pathoutputcommitter.factory.class"
          Hide
          ehiggs Ewan Higgs added a comment -

          Hi, I have been testing this locally with the Hortonworks cloud-integration project and an S3 compatible backend that is has strong consistency. Because it has strong consistency one would expect the NullMetadataStore to work. However, I'm getting some errors.

          To reproduce, I build Hadoop as follows:

          mvn install -DskipShade -Dmaven.javadoc.skip=true -Pdist,parallel-tests -DtestsThreadCount=8 -Djava.awt.headless=true -Ddeclared.hadoop.version=2.11 -DskipTests
          

          I ran into some NPEs:

          S3AFileStatus{path=s3a://s3guard-test/cloud-integration/DELAY_LISTING_ME/S3ACommitDataframeSuite/dataframe-committer/committer-default-orc/orc/part-00000-8b8b323b-c747-4d72-b331-b6de1c1f8387-c000.snappy.orc; isDirectory=false; length=2995; replication=1; blocksize=1048576; modification_time=1502715661000; access_time=0; owner=ehiggs; group=ehiggs; permission=rw-rw-rw-; isSymlink=false; hasAcl=false; isEncrypted=false; isErasureCoded=false} isEmptyDirectory=FALSE
          2017-08-14 15:01:02,297 [ScalaTest-main-running-S3ACommitDataframeSuite] DEBUG s3a.S3AFileSystem (Listing.java:sourceHasNext(374)) - Start iterating the provided status.
          2017-08-14 15:01:02,304 [ScalaTest-main-running-S3ACommitDataframeSuite] ERROR commit.S3ACommitDataframeSuite (Logging.scala:logError(91)) - After 237,747,296 nS: java.lang.NullPointerException
          java.lang.NullPointerException
          	at org.apache.hadoop.fs.LocatedFileStatus.<init>(LocatedFileStatus.java:87)
          	at org.apache.spark.sql.execution.datasources.InMemoryFileIndex$$anonfun$org$apache$spark$sql$execution$datasources$InMemoryFileIndex$$listLeafFiles$3.apply(InMemoryFileIndex.scala:299)
          	at org.apache.spark.sql.execution.datasources.InMemoryFileIndex$$anonfun$org$apache$spark$sql$execution$datasources$InMemoryFileIndex$$listLeafFiles$3.apply(InMemoryFileIndex.scala:281)
          	at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
          	at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
          	at scala.collection.IndexedSeqOptimized$class.foreach(IndexedSeqOptimized.scala:33)
          	at scala.collection.mutable.ArrayOps$ofRef.foreach(ArrayOps.scala:186)
          	at scala.collection.TraversableLike$class.map(TraversableLike.scala:234)
          	at scala.collection.mutable.ArrayOps$ofRef.map(ArrayOps.scala:186)
          	at org.apache.spark.sql.execution.datasources.InMemoryFileIndex$.org$apache$spark$sql$execution$datasources$InMemoryFileIndex$$listLeafFiles(InMemoryFileIndex.scala:281)
          	at org.apache.spark.sql.execution.datasources.InMemoryFileIndex$$anonfun$org$apache$spark$sql$execution$datasources$InMemoryFileIndex$$bulkListLeafFiles$1.apply(InMemoryFileIndex.scala:172)
          	at org.apache.spark.sql.execution.datasources.InMemoryFileIndex$$anonfun$org$apache$spark$sql$execution$datasources$InMemoryFileIndex$$bulkListLeafFiles$1.apply(InMemoryFileIndex.scala:171)
          	at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
          

          I'll attach the trace.

          Show
          ehiggs Ewan Higgs added a comment - Hi, I have been testing this locally with the Hortonworks cloud-integration project and an S3 compatible backend that is has strong consistency. Because it has strong consistency one would expect the NullMetadataStore to work. However, I'm getting some errors. To reproduce, I build Hadoop as follows: mvn install -DskipShade -Dmaven.javadoc.skip= true -Pdist,parallel-tests -DtestsThreadCount=8 -Djava.awt.headless= true -Ddeclared.hadoop.version=2.11 -DskipTests I ran into some NPEs: S3AFileStatus{path=s3a: //s3guard-test/cloud-integration/DELAY_LISTING_ME/S3ACommitDataframeSuite/dataframe-committer/committer- default -orc/orc/part-00000-8b8b323b-c747-4d72-b331-b6de1c1f8387-c000.snappy.orc; isDirectory= false ; length=2995; replication=1; blocksize=1048576; modification_time=1502715661000; access_time=0; owner=ehiggs; group=ehiggs; permission=rw-rw-rw-; isSymlink= false ; hasAcl= false ; isEncrypted= false ; isErasureCoded= false } isEmptyDirectory=FALSE 2017-08-14 15:01:02,297 [ScalaTest-main-running-S3ACommitDataframeSuite] DEBUG s3a.S3AFileSystem (Listing.java:sourceHasNext(374)) - Start iterating the provided status. 2017-08-14 15:01:02,304 [ScalaTest-main-running-S3ACommitDataframeSuite] ERROR commit.S3ACommitDataframeSuite (Logging.scala:logError(91)) - After 237,747,296 nS: java.lang.NullPointerException java.lang.NullPointerException at org.apache.hadoop.fs.LocatedFileStatus.<init>(LocatedFileStatus.java:87) at org.apache.spark.sql.execution.datasources.InMemoryFileIndex$$anonfun$org$apache$spark$sql$execution$datasources$InMemoryFileIndex$$listLeafFiles$3.apply(InMemoryFileIndex.scala:299) at org.apache.spark.sql.execution.datasources.InMemoryFileIndex$$anonfun$org$apache$spark$sql$execution$datasources$InMemoryFileIndex$$listLeafFiles$3.apply(InMemoryFileIndex.scala:281) at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234) at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234) at scala.collection.IndexedSeqOptimized$class.foreach(IndexedSeqOptimized.scala:33) at scala.collection.mutable.ArrayOps$ofRef.foreach(ArrayOps.scala:186) at scala.collection.TraversableLike$class.map(TraversableLike.scala:234) at scala.collection.mutable.ArrayOps$ofRef.map(ArrayOps.scala:186) at org.apache.spark.sql.execution.datasources.InMemoryFileIndex$.org$apache$spark$sql$execution$datasources$InMemoryFileIndex$$listLeafFiles(InMemoryFileIndex.scala:281) at org.apache.spark.sql.execution.datasources.InMemoryFileIndex$$anonfun$org$apache$spark$sql$execution$datasources$InMemoryFileIndex$$bulkListLeafFiles$1.apply(InMemoryFileIndex.scala:172) at org.apache.spark.sql.execution.datasources.InMemoryFileIndex$$anonfun$org$apache$spark$sql$execution$datasources$InMemoryFileIndex$$bulkListLeafFiles$1.apply(InMemoryFileIndex.scala:171) at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234) I'll attach the trace.
          Hide
          ehiggs Ewan Higgs added a comment -

          Log for some NullPointerExceptions after "Start iterating the provided status".

          Show
          ehiggs Ewan Higgs added a comment - Log for some NullPointerExceptions after "Start iterating the provided status".
          Hide
          stevel@apache.org Steve Loughran added a comment -

          thx. I'll look @ this.

          Show
          stevel@apache.org Steve Loughran added a comment - thx. I'll look @ this.
          Hide
          stevel@apache.org Steve Loughran added a comment -

          + another stack from Ewan (via email)

          However, there appears to be a race condition somewhere when I test this against our consistent S3 compatible object store when I’m using the NullMetadataStore. Usually the tests work but every e.g. 1/5 times I run it, I get the following error:
          - Dataframe+magic-orc *** FAILED ***
            org.apache.spark.SparkException: Job aborted.
            at org.apache.spark.sql.execution.datasources.FileFormatWriter$.write(FileFormatWriter.scala:206)
            at org.apache.spark.sql.execution.datasources.InsertIntoHadoopFsRelationCommand.run(InsertIntoHadoopFsRelationCommand.scala:152)
            at org.apache.spark.sql.execution.command.ExecutedCommandExec.sideEffectResult$lzycompute(commands.scala:80)
            at org.apache.spark.sql.execution.command.ExecutedCommandExec.sideEffectResult(commands.scala:75)
            at org.apache.spark.sql.execution.command.ExecutedCommandExec.doExecute(commands.scala:98)
            at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:112)
            at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:112)
            at org.apache.spark.sql.execution.SparkPlan$$anonfun$executeQuery$1.apply(SparkPlan.scala:133)
            at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)
            at org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:130)
            ...
            Cause: java.io.EOFException: s3a://s3guard-test/cloud-integration/DELAY_LISTING_ME/S3ACommitDataframeSuite/dataframe-committer/committer-magic-orc/orc/__magic/app-attempt-0000/task_20170809172117_0001_m_000000.pendingset: read finished prematurely
            at org.apache.hadoop.util.JsonSerialization.load(JsonSerialization.java:219)
            at org.apache.hadoop.util.JsonSerialization.load(JsonSerialization.java:189)
            at org.apache.hadoop.fs.s3a.commit.files.PendingSet.load(PendingSet.java:107)
            at org.apache.hadoop.fs.s3a.commit.AbstractS3GuardCommitter$5.run(AbstractS3GuardCommitter.java:493)
            at org.apache.hadoop.fs.s3a.commit.AbstractS3GuardCommitter$5.run(AbstractS3GuardCommitter.java:490)
            at org.apache.hadoop.fs.s3a.commit.Tasks$Builder$1.run(Tasks.java:252)
            at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
            at java.util.concurrent.FutureTask.run(FutureTask.java:266)
            at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
            at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
           
          Show
          stevel@apache.org Steve Loughran added a comment - + another stack from Ewan (via email) However, there appears to be a race condition somewhere when I test this against our consistent S3 compatible object store when I’m using the NullMetadataStore. Usually the tests work but every e.g. 1/5 times I run it, I get the following error: - Dataframe+magic-orc *** FAILED *** org.apache.spark.SparkException: Job aborted. at org.apache.spark.sql.execution.datasources.FileFormatWriter$.write(FileFormatWriter.scala:206) at org.apache.spark.sql.execution.datasources.InsertIntoHadoopFsRelationCommand.run(InsertIntoHadoopFsRelationCommand.scala:152) at org.apache.spark.sql.execution.command.ExecutedCommandExec.sideEffectResult$lzycompute(commands.scala:80) at org.apache.spark.sql.execution.command.ExecutedCommandExec.sideEffectResult(commands.scala:75) at org.apache.spark.sql.execution.command.ExecutedCommandExec.doExecute(commands.scala:98) at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:112) at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:112) at org.apache.spark.sql.execution.SparkPlan$$anonfun$executeQuery$1.apply(SparkPlan.scala:133) at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151) at org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:130) ... Cause: java.io.EOFException: s3a: //s3guard-test/cloud-integration/DELAY_LISTING_ME/S3ACommitDataframeSuite/dataframe-committer/committer-magic-orc/orc/__magic/app-attempt-0000/task_20170809172117_0001_m_000000.pendingset: read finished prematurely at org.apache.hadoop.util.JsonSerialization.load(JsonSerialization.java:219) at org.apache.hadoop.util.JsonSerialization.load(JsonSerialization.java:189) at org.apache.hadoop.fs.s3a.commit.files.PendingSet.load(PendingSet.java:107) at org.apache.hadoop.fs.s3a.commit.AbstractS3GuardCommitter$5.run(AbstractS3GuardCommitter.java:493) at org.apache.hadoop.fs.s3a.commit.AbstractS3GuardCommitter$5.run(AbstractS3GuardCommitter.java:490) at org.apache.hadoop.fs.s3a.commit.Tasks$Builder$1.run(Tasks.java:252) at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) at java.util.concurrent.FutureTask.run(FutureTask.java:266) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
          Hide
          stevel@apache.org Steve Loughran added a comment -

          Patch 036

          this is the committer applied to trunk now that s3guard is in

          Show
          stevel@apache.org Steve Loughran added a comment - Patch 036 this is the committer applied to trunk now that s3guard is in
          Hide
          stevel@apache.org Steve Loughran added a comment -

          HADOOP-13786 HADOOP-14531 lambda wrapper around all production s3 calls

          • all invocations of s3 calls are wrapped where appropriate, either with once() (which does the translation), retry() or retryUntranslated
          • javadocs state retry policy; this is propagated to give callers an idea of what retries already
          • commit tests -> java 8 lambdas too
          • test json serdeser in hadoop common
          • checkstyle

          the error handling includes improvement to translateexception to recognise dynamoDB throttling and also that json parse error which means an EOF on response parsing (which means, as its after-execution, that non-idempotent calls wont retry).

          The commit methods have been resilient to failures via the S3Lambda for a while, now that it's extended to all of them we can add methods to do fault injection on all operations: the retly logic in S3ARetryPolicy assumes that throttling (503), server error (500) and connection setup failures are always retryable. Therefore, if the client code is done right, you could run all the system tests with the injecting client set to throttle a limited percent of time.

          Although developed in the committer, we could tease this out (along with the moved WriteOperationsHelper) and add it to trunk standalone. That'd reduce the size of the HADOOP-13786 diff, and provide a single large patch for people to cherry pick. Though if they want to backport to branch-2 they get to convert every single lambda-exp into a callable, which, even though IDEA Can automate, will make for uglier code than:

              S3Object object = invoke.retry(text, uri, true,
                  () -> client.getObject(request));
          

          Anyway, I plan to continue with dev & test of the error handling in the committer branch, which, after all, depends on resilience of all its operations, even in the presence of transient failures. Once its stable it'd be something to pull out and get in standalone

          Show
          stevel@apache.org Steve Loughran added a comment - HADOOP-13786 HADOOP-14531 lambda wrapper around all production s3 calls all invocations of s3 calls are wrapped where appropriate, either with once() (which does the translation), retry() or retryUntranslated javadocs state retry policy; this is propagated to give callers an idea of what retries already commit tests -> java 8 lambdas too test json serdeser in hadoop common checkstyle the error handling includes improvement to translateexception to recognise dynamoDB throttling and also that json parse error which means an EOF on response parsing (which means, as its after-execution, that non-idempotent calls wont retry). The commit methods have been resilient to failures via the S3Lambda for a while, now that it's extended to all of them we can add methods to do fault injection on all operations: the retly logic in S3ARetryPolicy assumes that throttling (503), server error (500) and connection setup failures are always retryable. Therefore, if the client code is done right, you could run all the system tests with the injecting client set to throttle a limited percent of time. Although developed in the committer, we could tease this out (along with the moved WriteOperationsHelper) and add it to trunk standalone. That'd reduce the size of the HADOOP-13786 diff, and provide a single large patch for people to cherry pick. Though if they want to backport to branch-2 they get to convert every single lambda-exp into a callable, which, even though IDEA Can automate, will make for uglier code than: S3Object object = invoke.retry(text, uri, true , () -> client.getObject(request)); Anyway, I plan to continue with dev & test of the error handling in the committer branch, which, after all, depends on resilience of all its operations, even in the presence of transient failures. Once its stable it'd be something to pull out and get in standalone
          Hide
          stevel@apache.org Steve Loughran added a comment - - edited

          Patch 038;

          1. in sync with trunk now that MAPREDUCE-6956 starts to give us the committer flexibility in FileOutputFormat we need.
          2. test executed with throttling ramped up to 50% of s3 requests; tuned tests and inconsistent client for this to all work

          One thing I've done to help myself and other understand what the retry policy is for methods in and near s3aFS is to add a source-time-only annotation, @Retries, which I've used to

            @Retries.Retry_translated
            private void createFakeDirectory(final String objectName)
                throws IOException {
              if (!objectName.endsWith("/")) {
                createEmptyObject(objectName + "/");
              } else {
                createEmptyObject(objectName);
              }
            }
          

          That says: something underneath is retrying and translating. Which tells callers: don't bother wrapping it. In contrast, this says once with no translation

            @Retries.Once_raw
            InitiateMultipartUploadResult initiateMultipartUpload(
                InitiateMultipartUploadRequest request) throws IOException {
              LOG.debug("Initiate multipart upload to {}", request.getKey());
              incrementStatistic(OBJECT_MULTIPART_UPLOAD_INITIATED);
              return getAmazonS3Client().initiateMultipartUpload(request);
            }
          

          With the tags its easier to review the code, to make sure that between all low-level once_untranslated methods and those of the public APIs, you have retry and exception translation. I like this for seeing what's happening: it's clearer than just a comment in the javadocs, as it forces you to say what's going on. It's all just declarative, nothing to actually read the tags (e.g no javadoc plugin), and nothing in the binaries. Maybe we should add comments to the javadocs unless I can actually automate the generation (which would be very slick)

          tested: yes, ireland and frankfurt, as noted 50% error rate. Things doing rename() fail, because I haven't gone near the retry logic there. But as the new committers don't do renames, they are happy

          Show
          stevel@apache.org Steve Loughran added a comment - - edited Patch 038; in sync with trunk now that MAPREDUCE-6956 starts to give us the committer flexibility in FileOutputFormat we need. test executed with throttling ramped up to 50% of s3 requests; tuned tests and inconsistent client for this to all work One thing I've done to help myself and other understand what the retry policy is for methods in and near s3aFS is to add a source-time-only annotation, @Retries, which I've used to @Retries.Retry_translated private void createFakeDirectory( final String objectName) throws IOException { if (!objectName.endsWith( "/" )) { createEmptyObject(objectName + "/" ); } else { createEmptyObject(objectName); } } That says: something underneath is retrying and translating. Which tells callers: don't bother wrapping it. In contrast, this says once with no translation @Retries.Once_raw InitiateMultipartUploadResult initiateMultipartUpload( InitiateMultipartUploadRequest request) throws IOException { LOG.debug( "Initiate multipart upload to {}" , request.getKey()); incrementStatistic(OBJECT_MULTIPART_UPLOAD_INITIATED); return getAmazonS3Client().initiateMultipartUpload(request); } With the tags its easier to review the code, to make sure that between all low-level once_untranslated methods and those of the public APIs, you have retry and exception translation. I like this for seeing what's happening: it's clearer than just a comment in the javadocs, as it forces you to say what's going on. It's all just declarative, nothing to actually read the tags (e.g no javadoc plugin), and nothing in the binaries. Maybe we should add comments to the javadocs unless I can actually automate the generation (which would be very slick) tested: yes, ireland and frankfurt, as noted 50% error rate. Things doing rename() fail, because I haven't gone near the retry logic there. But as the new committers don't do renames, they are happy
          Hide
          stevel@apache.org Steve Loughran added a comment -

          for watches of this, SPARK-22217 fixes Spark's Parquet output commit protocol to switch to his (or any other non-Parquet/ FileOutputCommitter .

          There's a pull request which would welcome support.

          Show
          stevel@apache.org Steve Loughran added a comment - for watches of this, SPARK-22217 fixes Spark's Parquet output commit protocol to switch to his (or any other non-Parquet/ FileOutputCommitter . There's a pull request which would welcome support.
          Hide
          fabbri Aaron Fabbri added a comment -

          Thanks Steve Loughran. I'll bug any Spark folks that I know.

          Show
          fabbri Aaron Fabbri added a comment - Thanks Steve Loughran . I'll bug any Spark folks that I know.
          Hide
          stevel@apache.org Steve Loughran added a comment -

          Patch 039

          I've not pushed up a patch for a while, but been doing it on github with Ewan Higgs testing it too. I can now confirm that I have, locally, finished off the hardest bit of the downstream tests: Spark outputting partitioned Parquet data.

          With that working I'm happy there's no fundamental flaws in the design and in how it is bonding to committers.

          I want to get this in now; it's pretty much done other than docs. Reviews welcome!

          Testing: s3 ireland with/without dynamo, scale on. All is well with the committers

          • one transient s3guard CLI; filed separately
          • one 500 server error on a getFileStatusV1 call which the new retry logic didn't handle. There's no retry policy on a 500 request right now...we may want to evolve one. But that's related to error handling, not the commit code itself.
          Show
          stevel@apache.org Steve Loughran added a comment - Patch 039 I've not pushed up a patch for a while, but been doing it on github with Ewan Higgs testing it too. I can now confirm that I have, locally, finished off the hardest bit of the downstream tests: Spark outputting partitioned Parquet data. With that working I'm happy there's no fundamental flaws in the design and in how it is bonding to committers. I want to get this in now; it's pretty much done other than docs. Reviews welcome! Testing: s3 ireland with/without dynamo, scale on. All is well with the committers one transient s3guard CLI; filed separately one 500 server error on a getFileStatusV1 call which the new retry logic didn't handle. There's no retry policy on a 500 request right now...we may want to evolve one. But that's related to error handling, not the commit code itself.
          Hide
          fabbri Aaron Fabbri added a comment -

          Wow great work. I will re-review and do more testing on v39. It will take me some time just given the size of the patch (almost 1 MB).

          Show
          fabbri Aaron Fabbri added a comment - Wow great work. I will re-review and do more testing on v39. It will take me some time just given the size of the patch (almost 1 MB).
          Hide
          rdblue Ryan Blue added a comment -

          I'll try to take a look as well.

          Show
          rdblue Ryan Blue added a comment - I'll try to take a look as well.
          Hide
          stevel@apache.org Steve Loughran added a comment -
          1. I've just purged all the Yetus reviews to keep this patch smaller; I'll do another patch with retry logic added for a 500 response from s3/dynamo, in both cases assuming that a call can be re-issued. That is, even for something we don't consider idempotent (GET,...), if the server sent 500 back, you can still try again.
          Show
          stevel@apache.org Steve Loughran added a comment - I've just purged all the Yetus reviews to keep this patch smaller; I'll do another patch with retry logic added for a 500 response from s3/dynamo, in both cases assuming that a call can be re-issued. That is, even for something we don't consider idempotent (GET,...), if the server sent 500 back, you can still try again.
          Hide
          ehiggs Ewan Higgs added a comment -

          I've been testing this mostly from a performance point of view using Hadoop MR2 using NullMetadataStore and I'm pretty happy with the results. It's indeed twice as fast as the old style FileOutputCommitter on the system I used.

          There's a lot of code here and it's been moving quite quickly but it's in overall good shape, imo. As I'm using a NullMetadataStore a lot of the possible error scenarios won't popup for me so it will be great if people can cover those areas.

          Show
          ehiggs Ewan Higgs added a comment - I've been testing this mostly from a performance point of view using Hadoop MR2 using NullMetadataStore and I'm pretty happy with the results. It's indeed twice as fast as the old style FileOutputCommitter on the system I used. There's a lot of code here and it's been moving quite quickly but it's in overall good shape, imo. As I'm using a NullMetadataStore a lot of the possible error scenarios won't popup for me so it will be great if people can cover those areas.
          Hide
          stevel@apache.org Steve Loughran added a comment -

          Thanks (I should add that AFAIK Ewan has been testing the "magic" committer, not the staging ones; his store is naturally consistent.

          What I'd like to suggest here is we create a branch for the S3Guard phase II work (HADOOP-14825), make this the first commit & then work on the s3guard II improvements above it. That way: those of us working on S3 things have time to use all of this code before making the leap to say "ready for trunk", and we can avoid the problem of other patches to S3A conflicting with this one.

          Show
          stevel@apache.org Steve Loughran added a comment - Thanks (I should add that AFAIK Ewan has been testing the "magic" committer, not the staging ones; his store is naturally consistent. What I'd like to suggest here is we create a branch for the S3Guard phase II work ( HADOOP-14825 ), make this the first commit & then work on the s3guard II improvements above it. That way: those of us working on S3 things have time to use all of this code before making the leap to say "ready for trunk", and we can avoid the problem of other patches to S3A conflicting with this one.
          Hide
          ehiggs Ewan Higgs added a comment -

          I should add that AFAIK Ewan has been testing the "magic" committer, not the staging ones; his store is naturally consistent.

          Yes.

          What I'd like to suggest here is we create a branch for the S3Guard phase II work (HADOOP-14825), make this the first commit & then work on the s3guard II improvements above it.

          +1

          Show
          ehiggs Ewan Higgs added a comment - I should add that AFAIK Ewan has been testing the "magic" committer, not the staging ones; his store is naturally consistent. Yes. What I'd like to suggest here is we create a branch for the S3Guard phase II work ( HADOOP-14825 ), make this the first commit & then work on the s3guard II improvements above it. +1
          Hide
          fabbri Aaron Fabbri added a comment -

          I'm still reviewing and testing this stuff. Looks pretty good but takes time to cover 26,000 line diff with any rigor. My finding-bugs-by-inspection rate has been pretty low so far, congrats.

          What I'd like to suggest here is we create a branch for the S3Guard phase II work (HADOOP-14825), make this the first commit & then work on the s3guard II improvements above it.

          I appreciate the awesome work here. My two cents, taking a step back a bit: I think we should try to move towards small patches and short-lived feature branches. How long do we expect the feature branch to live? Two to four weeks is reasonable IMO. 2 is better.

          I'd like to make the case for keeping the main codepaths solid and integrated and feature-flag (config) new work, instead of having major rewrites living outside trunk for too long. Two main reasons: (1) Not blocking other work (2) better quality / less risk.. as we approach continuous integration we get quality benefits. Happy to elaborate on that if needed.

          The fantastic work folks like Steve Loughran have done on improving our tests really make this possible. We should take advantage of it.

          Show
          fabbri Aaron Fabbri added a comment - I'm still reviewing and testing this stuff. Looks pretty good but takes time to cover 26,000 line diff with any rigor. My finding-bugs-by-inspection rate has been pretty low so far, congrats. What I'd like to suggest here is we create a branch for the S3Guard phase II work ( HADOOP-14825 ), make this the first commit & then work on the s3guard II improvements above it. I appreciate the awesome work here. My two cents, taking a step back a bit: I think we should try to move towards small patches and short-lived feature branches. How long do we expect the feature branch to live? Two to four weeks is reasonable IMO. 2 is better. I'd like to make the case for keeping the main codepaths solid and integrated and feature-flag (config) new work, instead of having major rewrites living outside trunk for too long. Two main reasons: (1) Not blocking other work (2) better quality / less risk.. as we approach continuous integration we get quality benefits. Happy to elaborate on that if needed. The fantastic work folks like Steve Loughran have done on improving our tests really make this possible. We should take advantage of it.
          Hide
          stevel@apache.org Steve Loughran added a comment -

          I recognise the fear of long-lived branches, and we can probably get away with incremental s3guard stuff from now now. This committer is mostly done apart from some docs & tuning of related bits. The sole current diff since this patch and my local source is a new AWSStatus500Exception and treating a 500 response as retriable, even on ops considered non-idempotent, and some more docs on mapreduce task commit/abort.

          For review, you can split off looking at

          1. the retry logic: Invoker, S3ARetryPolicy & how that's being used to wrap operations in S3AFileSystem, a refactored WriteOperationsHelper and DynamoDBMetadataStore. Is the model write (retry-around-closures), is the retry policy good, and is it being used correctly,
          2. Changes to S3ABlockOutputStream to let us control whether its delayed complete or not, & changes to S3AFS to recognise the special paths so switch policy
          3. CommitOperations: the underlying integration with the FS to save/restore lists of PUTs to complete, operations to commit them

          Finally, the committer, looking at AbstractS3GuardCommitter , StagingCommitter (Ryan's), and MagicS3GuardCommitter which is the one using the special output streams. They all use the same bindings to the FS and JSON file formats, so differ in: where work goes, how the commit metadata is passed to the job committer. And for the staging committer, the conflict policies of the two public implementations, :Directory and Partitioned.

          The test AbstractITCommitProtocol is the one which pushes the commit protocol through its lifecycle, trying to recreate the valid & failure workflows. That's inevitably where there's scope to cover all the corner cases...I think I'll look again at speculation there.

          Finally, new docs, including one on committer architecture. That covers what the MR commit protocol is, which is something you need to understand before looking at the commit internals. That doc is probably the most complete discussion on the topic there is, and even it avoids bits I don't understand (Preemption)

          I can give talk on this stuff on wednesday or thursday AM PST if people want

          Show
          stevel@apache.org Steve Loughran added a comment - I recognise the fear of long-lived branches, and we can probably get away with incremental s3guard stuff from now now. This committer is mostly done apart from some docs & tuning of related bits. The sole current diff since this patch and my local source is a new AWSStatus500Exception and treating a 500 response as retriable, even on ops considered non-idempotent, and some more docs on mapreduce task commit/abort. For review, you can split off looking at the retry logic: Invoker , S3ARetryPolicy & how that's being used to wrap operations in S3AFileSystem, a refactored WriteOperationsHelper and DynamoDBMetadataStore. Is the model write (retry-around-closures), is the retry policy good, and is it being used correctly, Changes to S3ABlockOutputStream to let us control whether its delayed complete or not, & changes to S3AFS to recognise the special paths so switch policy CommitOperations: the underlying integration with the FS to save/restore lists of PUTs to complete, operations to commit them Finally, the committer, looking at AbstractS3GuardCommitter , StagingCommitter (Ryan's), and MagicS3GuardCommitter which is the one using the special output streams. They all use the same bindings to the FS and JSON file formats, so differ in: where work goes, how the commit metadata is passed to the job committer. And for the staging committer, the conflict policies of the two public implementations, :Directory and Partitioned. The test AbstractITCommitProtocol is the one which pushes the commit protocol through its lifecycle, trying to recreate the valid & failure workflows. That's inevitably where there's scope to cover all the corner cases...I think I'll look again at speculation there. Finally, new docs, including one on committer architecture . That covers what the MR commit protocol is, which is something you need to understand before looking at the commit internals. That doc is probably the most complete discussion on the topic there is, and even it avoids bits I don't understand (Preemption) I can give talk on this stuff on wednesday or thursday AM PST if people want
          Hide
          fabbri Aaron Fabbri added a comment -

          Going to start posting some comments on v39 megapatch. I still have to go back over some of the more involved parts of the code.

          +++ b/hadoop-common-project/hadoop-common/dev-support/findbugsExcludeFile.xml
          ...
          +  <!-- findbugs is mistaken -->
          +  <Match>
          +    <Class name="org.apache.hadoop.util.JsonSerializationy"/>
          

          Elaboration would be nice. Also I think you've misspelled the class name.

          + * This method was copied from
          + * {@code org.apache.hadoop.registry.client.binding.JsonSerDeser}.
          + * @param <T> Type to marshal.
          

          Can you explain why copy/paste versus import?

          +
          +<property>
          +  <name>fs.s3a.retry.throttle.interval</name>
          +  <value>1000ms</value>
          

          Interesting default. Any reasoning? Feels high, subjectively, but that depends on how throttling responses arrive at nodes in a cluster. If there is a large batch of them spread across nodes I'd think smaller interval + randomization would behave better (versus thundering herd after full second of silence).

          +  public static PathOutputCommitterFactory getCommitterFactory(
          ...
          +      if (StringUtils.isNotEmpty(conf.getTrimmed(schemeKey))) {
          ...
          +        key = schemeKey;
          +      }
          +    } else {
          +      // Either there's an explicit committer factory declared, or
          +      // there's no output path, so trying to do schema-specific lookup
          +      // cannot work.
          +    }
          

          Empty else block. Findbugs doesn't complain? Anyways you could just start your comment with "Else: Either..." versus using actual syntax?

          +  /**
          +   * Verify that if the committer factory class is unknown, you cannot
          +   * create committers.
          +   */
          +  @Test
          +  public void testCommitterNullOutputPath() throws Throwable {
          

          Update comment: "Verify that if the output path is null..."

          +  /**
          +   * Number of times to retry any repeatable S3 client request on failure,
          +   * excluding throttling requests: {@value}.
          +   */
          +  public static final String RETRY_LIMIT = "fs.s3a.retry.limit";
          

          Do we want to explain the difference from fs.s3a.attempts.maximum somewhere to avoid confusion?

          +  public <T> T retryUntranslated(
          ...
          +      try {
          +        // execute the operation, returning if successful
          +        return operation.execute();
          +      } catch (IOException | SdkBaseException e) {
          +        caught = e;
          +      }
          +      // you only get here if the operation didn't complete
          +      // normally, hence caught != null
          

          Unless exception e is not the caught type. What if SDK bug yields NPE, for example? Then you NPE in your code in translateException() guts, no?

          +public class Retries {
          

          Retries stuff great to see, but the change is invasive and needs separate careful consideration IMO. How painful would this be to pull out into separate patch?

          First question here is, what about SDK retries? I know the SDK documentation and behavior is spotty at best, but there are some known things (i.e. DynamoDB does retry in SDK for most operations, exclusive of batched operations which are application responsibility).

          +
          +      // does the output stream have delayed visibility
          +    case CommitConstants.STREAM_CAPABILITY_MAGIC_OUTPUT:
          +      return !putTracker.outputImmediatelyVisible();
          

          StreamCapabilites is a nice addition and works well here IMO.

          +   * This declared delete as idempotent.
          +   * This is an "interesting" topic in past Hadoop FS work.
          +   * Essentially: with a single caller, DELETE is idempotent
          +   * but in a shared filesystem, it is is very much not so.
          +   * Here, on the basis that isn't a filesystem with consistency guarantees,
          +   * retryable results in files being deleted.
          +  */
          +  private static final boolean DELETE_CONSIDERED_IDEMPOTENT = true;
          

          Yes, interesting. Is the argument that delete is racy anyways, even if clients externally synchronize, thus we treat it as idempotent even though it is not?
          Anyways, retry on failure can lead to success, but retry after (a) a request that did end up deleting it or (b) after a different client deleted from under us will cause an error after retries are exhausted (since subsequent retries
          will always fail to delete). Do we want a different retry policy for this sort of thing that is not-really-idempotent-but-we-retry-anyways?

          +        abortOutstandingMultipartUploads(purgeExistingMultipartAge);
          +      } catch (AccessDeniedException e) {
          +        instrumentation.errorIgnored();
          +        LOG.debug("Failed to purging multipart uploads against {}," +
          +            " FS may be read only", bucket, e);
          

          LOG.info()? I feel like folks might want to know their purge policy is not working. You can always shut it up with per-bucket config.

          +  @Retries.RetryRaw
          

          I like these annotations. Pretty much necessary to reason about the surface
          area / scope around retries and translations. Now if we could get the same for the SDK, please.

          + * Although the committer APIs allow for a committer to be created without
          + * an output path, this is no supported in this class or its subclasses:
          

          /no/not/

          +public final class Tasks {
          

          Didn't run this through the wet compiler carefully yet. Are there unit tests somewhere I didn't notice?

          +    try (DurationInfo d = new DurationInfo("%s: aborting job in state %s ",
          +        r, CommitUtils.jobIdString(context), state)) {
          

          How about passing a Logger into DurationInfo instead of it using its own?

          +  public void cleanupStagingDirs() throws IOException {
          +
          +  }
          
          +  protected void cleanup(JobContext context, boolean suppressExceptions)
          +      throws IOException {
          +
          +  }
          

          Just curious, could these be abstract, or do you want a default no-op without making it explicit in subclass?

          +      throws IOException {
          +    if (pending == null || pending.isEmpty()) {
          +      LOG.info("{}: no pending commits to abort", getRole());
          +    } else {
          +      Tasks.foreach(pending)
          +          .throwFailureWhenFinished(!suppressExceptions)
          +          .executeWith(buildThreadPool(context))
          +          .onFailure((commit, exception) ->
          +              getCommitOperations().abortSingleCommit(commit))
          +          .run(commit -> getCommitOperations().abortSingleCommit(commit));
          

          Failure callback is same as intended operation. Is this in effect a single retry?

          +/**
          + * Dynamically create the output committer based on the filesystem type.
          + * For S3A output, uses the committer provided in the implementation class.
          + * For other filesystems, returns the classic committer.
          

          Does it? Looks like it throws a "FS not supported" exception.

          +  private void innerCommit(SinglePendingCommit commit) throws IOException {
          +    // finalize the commit
          +    writeOperations.completeMPUwithRetries(
          +        commit.getDestinationKey(),
          +              commit.getUploadId(),
          +              CommitUtils.toPartEtags(commit.getEtags()),
          +              commit.getLength(),
          +              new AtomicInteger(0));
          

          Curious: why an atomic here?

          +  /**
          +   * Robust list files.
          +   * @param path path
          +   * @param recursive recursive listing?
          +   * @return iterator (which is *not* robust)
          

          Why robust? Did you used to have retry logic here? Now it just calls underlying function. Same with the function below it.

          +public class DynamicCommitterFactory extends AbstractS3GuardCommitterFactory {
          

          General naming comment: Should these be called S3A Committers instead of S3Guard Committers, now that one of the main implementations is orthogonal to S3Guard? Same comment goes for the docs.

          +/**
          + * Path operations for the staging committers.
          + */
          +public final class Paths {
          

          Can you comment on test coverage for this class? I didn't see a unit test but may have missed it.

          -        res = dynamoDB.batchWriteItemUnprocessed(unprocessed);
          -        unprocessed = res.getUnprocessedItems();
          +        final Map<String, List<WriteRequest>> p = unprocessed;
          +        unprocessed = dataAccess.retry("batchWrite", "", true,
          +            () ->
          +              dynamoDB.batchWriteItemUnprocessed(p).getUnprocessedItems()
          +             );
          

          I think you want to remove the retry here. The existing function appears to work correctly for retries (HADOOP-13904) as they are "supposed" to work. You are retrying the retries inside of an existing retry loop but not retrying the original operation above it. I found it confusing.

          In general, especially in this dynamo class, it seems like we're covering things that are supposed to be retried by SDK with retries. I'm not sure the benefit of that versus the risk of breaking something or eliminating fail-fast behavior when it is warranted?

          Show
          fabbri Aaron Fabbri added a comment - Going to start posting some comments on v39 megapatch. I still have to go back over some of the more involved parts of the code. +++ b/hadoop-common-project/hadoop-common/dev-support/findbugsExcludeFile.xml ... + <!-- findbugs is mistaken --> + <Match> + <Class name="org.apache.hadoop.util.JsonSerializationy"/> Elaboration would be nice. Also I think you've misspelled the class name. + * This method was copied from + * {@code org.apache.hadoop.registry.client.binding.JsonSerDeser}. + * @param <T> Type to marshal. Can you explain why copy/paste versus import? + +<property> + <name>fs.s3a.retry.throttle.interval</name> + <value>1000ms</value> Interesting default. Any reasoning? Feels high, subjectively, but that depends on how throttling responses arrive at nodes in a cluster. If there is a large batch of them spread across nodes I'd think smaller interval + randomization would behave better (versus thundering herd after full second of silence). + public static PathOutputCommitterFactory getCommitterFactory( ... + if (StringUtils.isNotEmpty(conf.getTrimmed(schemeKey))) { ... + key = schemeKey; + } + } else { + // Either there's an explicit committer factory declared, or + // there's no output path, so trying to do schema-specific lookup + // cannot work. + } Empty else block. Findbugs doesn't complain? Anyways you could just start your comment with "Else: Either..." versus using actual syntax? + /** + * Verify that if the committer factory class is unknown, you cannot + * create committers. + */ + @Test + public void testCommitterNullOutputPath() throws Throwable { Update comment: "Verify that if the output path is null..." + /** + * Number of times to retry any repeatable S3 client request on failure, + * excluding throttling requests: {@value}. + */ + public static final String RETRY_LIMIT = "fs.s3a.retry.limit"; Do we want to explain the difference from fs.s3a.attempts.maximum somewhere to avoid confusion? + public <T> T retryUntranslated( ... + try { + // execute the operation, returning if successful + return operation.execute(); + } catch (IOException | SdkBaseException e) { + caught = e; + } + // you only get here if the operation didn't complete + // normally, hence caught != null Unless exception e is not the caught type. What if SDK bug yields NPE, for example? Then you NPE in your code in translateException() guts, no? +public class Retries { Retries stuff great to see, but the change is invasive and needs separate careful consideration IMO. How painful would this be to pull out into separate patch? First question here is, what about SDK retries? I know the SDK documentation and behavior is spotty at best, but there are some known things (i.e. DynamoDB does retry in SDK for most operations, exclusive of batched operations which are application responsibility). + + // does the output stream have delayed visibility + case CommitConstants.STREAM_CAPABILITY_MAGIC_OUTPUT: + return !putTracker.outputImmediatelyVisible(); StreamCapabilites is a nice addition and works well here IMO. + * This declared delete as idempotent. + * This is an "interesting" topic in past Hadoop FS work. + * Essentially: with a single caller, DELETE is idempotent + * but in a shared filesystem, it is is very much not so. + * Here, on the basis that isn't a filesystem with consistency guarantees, + * retryable results in files being deleted. + */ + private static final boolean DELETE_CONSIDERED_IDEMPOTENT = true; Yes, interesting. Is the argument that delete is racy anyways, even if clients externally synchronize, thus we treat it as idempotent even though it is not? Anyways, retry on failure can lead to success, but retry after (a) a request that did end up deleting it or (b) after a different client deleted from under us will cause an error after retries are exhausted (since subsequent retries will always fail to delete). Do we want a different retry policy for this sort of thing that is not-really-idempotent-but-we-retry-anyways? + abortOutstandingMultipartUploads(purgeExistingMultipartAge); + } catch (AccessDeniedException e) { + instrumentation.errorIgnored(); + LOG.debug("Failed to purging multipart uploads against {}," + + " FS may be read only", bucket, e); LOG.info()? I feel like folks might want to know their purge policy is not working. You can always shut it up with per-bucket config. + @Retries.RetryRaw I like these annotations. Pretty much necessary to reason about the surface area / scope around retries and translations. Now if we could get the same for the SDK, please. + * Although the committer APIs allow for a committer to be created without + * an output path, this is no supported in this class or its subclasses: /no/not/ +public final class Tasks { Didn't run this through the wet compiler carefully yet. Are there unit tests somewhere I didn't notice? + try (DurationInfo d = new DurationInfo("%s: aborting job in state %s ", + r, CommitUtils.jobIdString(context), state)) { How about passing a Logger into DurationInfo instead of it using its own? + public void cleanupStagingDirs() throws IOException { + + } + protected void cleanup(JobContext context, boolean suppressExceptions) + throws IOException { + + } Just curious, could these be abstract, or do you want a default no-op without making it explicit in subclass? + throws IOException { + if (pending == null || pending.isEmpty()) { + LOG.info("{}: no pending commits to abort", getRole()); + } else { + Tasks.foreach(pending) + .throwFailureWhenFinished(!suppressExceptions) + .executeWith(buildThreadPool(context)) + .onFailure((commit, exception) -> + getCommitOperations().abortSingleCommit(commit)) + .run(commit -> getCommitOperations().abortSingleCommit(commit)); Failure callback is same as intended operation. Is this in effect a single retry? +/** + * Dynamically create the output committer based on the filesystem type. + * For S3A output, uses the committer provided in the implementation class. + * For other filesystems, returns the classic committer. Does it? Looks like it throws a "FS not supported" exception. + private void innerCommit(SinglePendingCommit commit) throws IOException { + // finalize the commit + writeOperations.completeMPUwithRetries( + commit.getDestinationKey(), + commit.getUploadId(), + CommitUtils.toPartEtags(commit.getEtags()), + commit.getLength(), + new AtomicInteger(0)); Curious: why an atomic here? + /** + * Robust list files. + * @param path path + * @param recursive recursive listing? + * @return iterator (which is *not* robust) Why robust? Did you used to have retry logic here? Now it just calls underlying function. Same with the function below it. +public class DynamicCommitterFactory extends AbstractS3GuardCommitterFactory { General naming comment: Should these be called S3A Committers instead of S3Guard Committers, now that one of the main implementations is orthogonal to S3Guard? Same comment goes for the docs. +/** + * Path operations for the staging committers. + */ +public final class Paths { Can you comment on test coverage for this class? I didn't see a unit test but may have missed it. - res = dynamoDB.batchWriteItemUnprocessed(unprocessed); - unprocessed = res.getUnprocessedItems(); + final Map<String, List<WriteRequest>> p = unprocessed; + unprocessed = dataAccess.retry("batchWrite", "", true, + () -> + dynamoDB.batchWriteItemUnprocessed(p).getUnprocessedItems() + ); I think you want to remove the retry here. The existing function appears to work correctly for retries ( HADOOP-13904 ) as they are "supposed" to work. You are retrying the retries inside of an existing retry loop but not retrying the original operation above it. I found it confusing. In general, especially in this dynamo class, it seems like we're covering things that are supposed to be retried by SDK with retries. I'm not sure the benefit of that versus the risk of breaking something or eliminating fail-fast behavior when it is warranted?
          Hide
          fabbri Aaron Fabbri added a comment -

          Feeling another wave coming on..

          Documentation is awesome, good work. Funny that the thorough treatment of output commit is living here. We may want to link from MR site docs.

          +or it is at the destination, -in which case the rename actually successed.
          

          /successed/suceeded/. I do like "successed" though, colloquially.

          +1. Workers execute "Tasks", which are fractiona of the job, a job whose
          

          /fractiona/fractions/

          +This then is the problem which the S3Guard committers address: how to safely
          ...
          +## Meet the S3Guard Commmitters
          

          /S3Guard committers/S3A committers/ ? If so search/replace other spots.

          +attempts to write to specifci "magic" paths are interpreted as writes
          

          /specifci/specific/

          +committed, VMs using the committer do not need to so much storage
          +But the committer needs a "consistent" S3 store (via S3Guard when working with
          

          Capitalization / punctuation.

          +does meanthat the subsequent readers of work *will* need a consistent listing
          

          /meanthat/mean that/

          +All of the S3A committers revert to provide a classic FileOutputCommitter instance
          

          /provide/providing/

          +factories to be used to create a committer, based on the specific value of theoption `fs.s3a.committer.name`:
          

          /theoption/the option/

          +This committer an extension of the "Directory" committer which has a special conflict resolution
          

          /an/is an/

          +The Jjb is configured to use the magic committer, but the S3A bucket has not been explicitly
          

          /Jjb/job/

          +This can be done for those buckets which are known to be consistent, either
          +because the [S3Guard](s3guard.html) is used to provide consistency,
          

          append: "or because the S3-compatible filesystem is known to be strongly consistent"

          +protocol for the files listing the pending write operations. Tt uses
          

          /Tt/It/

          +One way to find out what is happening (i.e. get a stack trace of where the committer
          +is being created is to set the option `mapreduce.fileoutputcommitter.algorithm.version`
          +to a value such as "10".
          

          Nice trick.

          +There is an expectation that the Job Driver and tasks can communicate: if a task
          +perform any operations itself during the task commit phase, it shall only do
          

          /perform/performs/

          +* For each attempt, and attempt ID is created, to build the job attempt ID.
          

          /and attempt ID/an attempt ID/

          +during task the task commit algorithm. When working with object stores which
          

          remove first "task"

          + Historically then, it is the "v2 algorithm"
          

          /v2 algorithm/second version of an algorithm/.. found it confusing on first read. Think this helps.

          +Resource Manager with in `yarn.app.mapreduce.am.ob.committer.commit-window` milliseconds
          +(default: 10,000). It does this by waiting until the next heartbeat it received.
          +There's a s possible bug here: if the interval is set too small the thread may
          

          /a s/a/

          +permanently spin waiting a callback within the window. Ignoring that, this algorithm
          

          Interesting stuff. Also, besides not being able to begin commit within the window, on non-realtime OS / storage there is no bound on time between beginning the commit and actually doing it, technically.

          +The AM may call the `OutputCommitter.taskAbort` on with a task attempt context,
          

          /on with/?/

          +1. There is a PUT operation, capable of uploading 5GB of data in one HTTP request.
          +1. The PUT operation is atomic, but there is no PUT-no-overwrite option.
          

          And multiple clients racing to write same path cannot detect which one actually succeeded via return code (IIRC this is determined by S3 service-side timestamp and not visible to clients). Am I right or mistaken here?

          +metadata. Other implementations of the S3 protocol are fully consistent; the
          

          clarity nit: /implementations of the S3 protocol/S3 protocol-compatible storage systems/

          +The simplicity of the Stocator committer is somethign to appreciate.
          

          /somthign/something/

          +the server's local filesystem can add/update an entry the index table of the
          

          /can/and/

          +Ryan Blue, of Netflix, has submitted an alternate committer, one which has a
          

          /Ryan Blue/Ryan Blue, the Duke of Netflix and quite a fine gentleman whom we admire deeply/
          (Optional change.)

          +**Failure during task execution**
          +
          +All data is written to local temporary files; these need to be cleaned up.
          +
          +The job must ensure that the local (pending) data is purged. *TODO*: test this
          

          TODO here. Possible subtask for someone to help with once we have a feature branch?

          +**Failure to communicate with S3 during data upload**
          +
          +If an upload fails, tasks will
          +* attempt to abort PUT requests already uploaded to S3
          

          Mention retry policy here? dunno.

          +1. It requires a path element, such as `__magic` which cannot be used
          +for any purpose other than for the storage of pending commit data.
          

          Aside: We could make this string configurable in the future, no?

          +
          +### Changes to `S3ABlockOutputStream`
          +
          +We can avoid having to copy and past the `S3ABlockOutputStream` by
          

          /past/paste/
          Also, this sounds outdated vs. the code. May want to make a pass on some of the tentative-sounding stuff you wrote that has since been resolved.

          +Update: There is a cost to this: MRv1 support is lost wihtout
          

          typo at end of line.

          +Thr committers can only be tested against an S3-compatible object store.
          

          /Thr/The/

          +It has since been extended to collet metrics and other values, and has proven
          

          /collet/collect/

          +The short term solution of a dynamic wrapper committer could postpone the need for this.
          

          Section sounds like it needs updating.

          +The S3A client can ba configured to rety those operations which are considered
          

          /rety/retry/

          +paths can interfere with each other
          

          add period.

          +    skipDuringFaultInjection(fs);
          

          Nice.

          +log4j.logger.org.apache.hadoop.fs.s3a=DEBUG
          

          Seems a little heavy.

          Show
          fabbri Aaron Fabbri added a comment - Feeling another wave coming on.. Documentation is awesome, good work. Funny that the thorough treatment of output commit is living here. We may want to link from MR site docs. +or it is at the destination, -in which case the rename actually successed. /successed/suceeded/. I do like "successed" though, colloquially. +1. Workers execute "Tasks", which are fractiona of the job, a job whose /fractiona/fractions/ +This then is the problem which the S3Guard committers address: how to safely ... +## Meet the S3Guard Commmitters /S3Guard committers/S3A committers/ ? If so search/replace other spots. +attempts to write to specifci "magic" paths are interpreted as writes /specifci/specific/ +committed, VMs using the committer do not need to so much storage +But the committer needs a "consistent" S3 store (via S3Guard when working with Capitalization / punctuation. +does meanthat the subsequent readers of work *will* need a consistent listing /meanthat/mean that/ +All of the S3A committers revert to provide a classic FileOutputCommitter instance /provide/providing/ +factories to be used to create a committer, based on the specific value of theoption `fs.s3a.committer.name`: /theoption/the option/ +This committer an extension of the "Directory" committer which has a special conflict resolution /an/is an/ +The Jjb is configured to use the magic committer, but the S3A bucket has not been explicitly /Jjb/job/ +This can be done for those buckets which are known to be consistent, either +because the [S3Guard](s3guard.html) is used to provide consistency, append: "or because the S3-compatible filesystem is known to be strongly consistent" +protocol for the files listing the pending write operations. Tt uses /Tt/It/ +One way to find out what is happening (i.e. get a stack trace of where the committer +is being created is to set the option `mapreduce.fileoutputcommitter.algorithm.version` +to a value such as "10". Nice trick. +There is an expectation that the Job Driver and tasks can communicate: if a task +perform any operations itself during the task commit phase, it shall only do /perform/performs/ +* For each attempt, and attempt ID is created, to build the job attempt ID. /and attempt ID/an attempt ID/ +during task the task commit algorithm. When working with object stores which remove first "task" + Historically then, it is the "v2 algorithm" /v2 algorithm/second version of an algorithm/.. found it confusing on first read. Think this helps. +Resource Manager with in `yarn.app.mapreduce.am.ob.committer.commit-window` milliseconds +(default: 10,000). It does this by waiting until the next heartbeat it received. +There's a s possible bug here: if the interval is set too small the thread may /a s/a/ +permanently spin waiting a callback within the window. Ignoring that, this algorithm Interesting stuff. Also, besides not being able to begin commit within the window, on non-realtime OS / storage there is no bound on time between beginning the commit and actually doing it, technically. +The AM may call the `OutputCommitter.taskAbort` on with a task attempt context, /on with/?/ +1. There is a PUT operation, capable of uploading 5GB of data in one HTTP request. +1. The PUT operation is atomic, but there is no PUT-no-overwrite option. And multiple clients racing to write same path cannot detect which one actually succeeded via return code (IIRC this is determined by S3 service-side timestamp and not visible to clients). Am I right or mistaken here? +metadata. Other implementations of the S3 protocol are fully consistent; the clarity nit: /implementations of the S3 protocol/S3 protocol-compatible storage systems/ +The simplicity of the Stocator committer is somethign to appreciate. /somthign/something/ +the server's local filesystem can add/update an entry the index table of the /can/and/ +Ryan Blue, of Netflix, has submitted an alternate committer, one which has a /Ryan Blue/Ryan Blue, the Duke of Netflix and quite a fine gentleman whom we admire deeply/ (Optional change.) +**Failure during task execution** + +All data is written to local temporary files; these need to be cleaned up. + +The job must ensure that the local (pending) data is purged. *TODO*: test this TODO here. Possible subtask for someone to help with once we have a feature branch? +**Failure to communicate with S3 during data upload** + +If an upload fails, tasks will +* attempt to abort PUT requests already uploaded to S3 Mention retry policy here? dunno. +1. It requires a path element, such as `__magic` which cannot be used +for any purpose other than for the storage of pending commit data. Aside: We could make this string configurable in the future, no? + +### Changes to `S3ABlockOutputStream` + +We can avoid having to copy and past the `S3ABlockOutputStream` by /past/paste/ Also, this sounds outdated vs. the code. May want to make a pass on some of the tentative-sounding stuff you wrote that has since been resolved. +Update: There is a cost to this: MRv1 support is lost wihtout typo at end of line. +Thr committers can only be tested against an S3-compatible object store. /Thr/The/ +It has since been extended to collet metrics and other values, and has proven /collet/collect/ +The short term solution of a dynamic wrapper committer could postpone the need for this. Section sounds like it needs updating. +The S3A client can ba configured to rety those operations which are considered /rety/retry/ +paths can interfere with each other add period. + skipDuringFaultInjection(fs); Nice. +log4j.logger.org.apache.hadoop.fs.s3a=DEBUG Seems a little heavy.
          Hide
          fabbri Aaron Fabbri added a comment -

          In terms of testing, one issue I'm hitting on OS X is the ITMagicCommitMRJob test seems to hang. The output keeps complaining about du command failure (pasted below). Looking into it. Shout if you've seen anything similar.

          2017-10-18 15:58:58,914 [RM Event dispatcher] INFO  attempt.RMAppAttemptImpl (RMAppAttemptImpl.java:handle(918)) - appattempt_1508367534738_0001_000001 State change from NEW to SUBMITTED on event = START
          2017-10-18 15:58:58,930 [RM Event dispatcher] INFO  attempt.RMAppAttemptImpl (RMAppAttemptImpl.java:handle(918)) - appattempt_1508367534738_0001_000001 State change from SUBMITTED to SCHEDULED on event = ATTEMPT_ADDED
          2017-10-18 15:58:59,092 [JUnit-testMRJob] INFO  impl.YarnClientImpl (YarnClientImpl.java:submitApplication(295)) - Submitted application application_1508367534738_0001
          2017-10-18 15:58:59,117 [JUnit-testMRJob] INFO  mapreduce.Job (Job.java:submit(1574)) - The url to track the job: http://fabbri-laptop.lan:0/proxy/application_1508367534738_0001/
          2017-10-18 15:58:59,117 [JUnit-testMRJob] INFO  commit.DurationInfo (DurationInfo.java:<init>(44)) - Starting: Job Execution
          2017-10-18 15:58:59,118 [JUnit-testMRJob] INFO  mapreduce.Job (Job.java:monitorAndPrintJob(1619)) - Running job: job_1508367534738_0001
          2017-10-18 15:59:23,240 [Log Scanner/Cleaner #0] INFO  hs.JobHistory (JobHistory.java:run(201)) - History Cleaner started
          2017-10-18 15:59:23,255 [Log Scanner/Cleaner #0] INFO  hs.JobHistory (JobHistory.java:run(207)) - History Cleaner complete
          2017-10-18 16:01:53,232 [Log Scanner/Cleaner #1] INFO  hs.JobHistory (JobHistory.java:run(191)) - Starting scan to move intermediate done files
          2017-10-18 16:08:19,766 [refreshUsed-/Users/fabbri/Code/hadoop/hadoop-tools/hadoop-aws/target/test/data/dfs/data/data1/current/BP-135722307-192.168.0.110-1508367530352] WARN  fs.CachingGetSpaceUsed (DU.java:refresh(55)) - Could not get disk usage information for path /Users/fabbri/Code/hadoop/hadoop-tools/hadoop-aws/target/test/data/dfs/data/data1/current/BP-135722307-192.168.0.110-1508367530352
          java.io.IOException: Expecting a line not the end of stream
                  at org.apache.hadoop.fs.DU$DUShell.parseExecResult(DU.java:79)
                  at org.apache.hadoop.util.Shell.runCommand(Shell.java:980)
                  at org.apache.hadoop.util.Shell.run(Shell.java:887)
                  at org.apache.hadoop.fs.DU$DUShell.startRefresh(DU.java:62)
                  at org.apache.hadoop.fs.DU.refresh(DU.java:53)
                  at org.apache.hadoop.fs.CachingGetSpaceUsed$RefreshThread.run(CachingGetSpaceUsed.java:181)
                  at java.lang.Thread.run(Thread.java:745)
          
          Show
          fabbri Aaron Fabbri added a comment - In terms of testing, one issue I'm hitting on OS X is the ITMagicCommitMRJob test seems to hang. The output keeps complaining about du command failure (pasted below). Looking into it. Shout if you've seen anything similar. 2017-10-18 15:58:58,914 [RM Event dispatcher] INFO attempt.RMAppAttemptImpl (RMAppAttemptImpl.java:handle(918)) - appattempt_1508367534738_0001_000001 State change from NEW to SUBMITTED on event = START 2017-10-18 15:58:58,930 [RM Event dispatcher] INFO attempt.RMAppAttemptImpl (RMAppAttemptImpl.java:handle(918)) - appattempt_1508367534738_0001_000001 State change from SUBMITTED to SCHEDULED on event = ATTEMPT_ADDED 2017-10-18 15:58:59,092 [JUnit-testMRJob] INFO impl.YarnClientImpl (YarnClientImpl.java:submitApplication(295)) - Submitted application application_1508367534738_0001 2017-10-18 15:58:59,117 [JUnit-testMRJob] INFO mapreduce.Job (Job.java:submit(1574)) - The url to track the job: http://fabbri-laptop.lan:0/proxy/application_1508367534738_0001/ 2017-10-18 15:58:59,117 [JUnit-testMRJob] INFO commit.DurationInfo (DurationInfo.java:<init>(44)) - Starting: Job Execution 2017-10-18 15:58:59,118 [JUnit-testMRJob] INFO mapreduce.Job (Job.java:monitorAndPrintJob(1619)) - Running job: job_1508367534738_0001 2017-10-18 15:59:23,240 [Log Scanner/Cleaner #0] INFO hs.JobHistory (JobHistory.java:run(201)) - History Cleaner started 2017-10-18 15:59:23,255 [Log Scanner/Cleaner #0] INFO hs.JobHistory (JobHistory.java:run(207)) - History Cleaner complete 2017-10-18 16:01:53,232 [Log Scanner/Cleaner #1] INFO hs.JobHistory (JobHistory.java:run(191)) - Starting scan to move intermediate done files 2017-10-18 16:08:19,766 [refreshUsed-/Users/fabbri/Code/hadoop/hadoop-tools/hadoop-aws/target/test/data/dfs/data/data1/current/BP-135722307-192.168.0.110-1508367530352] WARN fs.CachingGetSpaceUsed (DU.java:refresh(55)) - Could not get disk usage information for path /Users/fabbri/Code/hadoop/hadoop-tools/hadoop-aws/target/test/data/dfs/data/data1/current/BP-135722307-192.168.0.110-1508367530352 java.io.IOException: Expecting a line not the end of stream at org.apache.hadoop.fs.DU$DUShell.parseExecResult(DU.java:79) at org.apache.hadoop.util.Shell.runCommand(Shell.java:980) at org.apache.hadoop.util.Shell.run(Shell.java:887) at org.apache.hadoop.fs.DU$DUShell.startRefresh(DU.java:62) at org.apache.hadoop.fs.DU.refresh(DU.java:53) at org.apache.hadoop.fs.CachingGetSpaceUsed$RefreshThread.run(CachingGetSpaceUsed.java:181) at java.lang.Thread.run(Thread.java:745)

            People

            • Assignee:
              stevel@apache.org Steve Loughran
              Reporter:
              stevel@apache.org Steve Loughran
            • Votes:
              10 Vote for this issue
              Watchers:
              36 Start watching this issue

              Dates

              • Created:
                Updated:

                Development