Details

    • Type: Sub-task
    • Status: Resolved
    • Priority: Major
    • Resolution: Fixed
    • Affects Version/s: 2.9.0
    • Fix Version/s: 2.8.0, 3.0.0-alpha2
    • Component/s: fs/s3
    • Labels:
      None
    • Target Version/s:
    • Hadoop Flags:
      Incompatible change
    • Release Note:
      This mechanism replaces the (experimental) fast output stream of Hadoop 2.7.x, combining better scalability options with instrumentation. Consult the S3A documentation to see the extra configuration operations.

      Description

      There's two output stream mechanisms in Hadooop 2.7.x, neither of which handle massive multi-GB files that well.

      "classic": buffer everything to HDD until to the close() operation; time to close becomes O(data); as is available disk space. Fails to exploit exploit idle bandwidth, and on EC2 VMs with not much HDD capacity (especially completing with HDFS storage), can fill up the disk.

      S3AFastOutputStream uploads data in partition-sized blocks, buffering via byte arrays. Avoids disk problems and as it writes as soon as the first partition is ready, close() time is O(outstanding-data). However: needs tuning to reduce amount of data buffered. Get it wrong, and the first clue you get may be that the process goes OOM or is killed by YARN. Which is a shame, as get it right and operations which generates lots of data, complete much faster, including distcp.

      This patch proposes a new output stream, a successor to both, S3ABlockOutputStream.

      1. uses block upload model of S3AFastOutputStream
      2. supports buffering via: HDD, heap and (recycled) byte buffer, offering a choice between memory and HDD use. HDD: no OOM problems on small JVMs/need to tune.
      3. Uses the fast output stream mechanism of limiting queue size for data to upload. Even when buffering via HDD, you may need to limit that use.
      4. lots of instrumentation to see what's being written.
      5. good defaults out the box (e.g buffer to HDD, partition size to strike a good balance of early upload and scaleability)
      6. robust against transient failures. The AWS SDK retries a PUT on failure; the entire block may need to be replayed, so HDD input cannot be buffered via java.io.BufferedInputStream. It has also surfaced in testing that if the final commit of a multipart option fails, it isn't retried —at least in the current SDK in use. Do that ourselves.
      7. use roundrobin directory allocation, for most effective disk use
      8. take an AWS SDK com.amazonaws.event.ProgressListener for progress callbacks, giving more detail on the operation. (It actually takes a org.apache.hadoop.util.Progressable, but if that also implements the AWS interface, that is used instead.

      All of this to come with scale tests

      • generate large files using all buffer mechanisms
      • Do a large copy/rname and verify that the copy really works, including metadata
      • be configurable with sizes up to muti-GB, which also means that the test timeouts need to be configurable to match the time it can take.
      • As they are slow, make them optional, using the -Dscale switch to enable.

      Verifying large file rename is important on its own, as it is needed for very large commit operations for committers using rename

      The goal here is to implement a single, object stream which can be used for all outputs, tuneable as to whether to use disk or memory, and on queue sizes, but otherwise be all that's needed. We can do future development on this, remove its predecessor S3AFastOutputStream, so keeping docs and testing down, and leave the original S3AOutputStream alone for regression testing/fallback.

      1. HADOOP-13560-branch-2-004.patch
        164 kB
        Steve Loughran
      2. HADOOP-13560-branch-2-003.patch
        161 kB
        Steve Loughran
      3. HADOOP-13560-branch-2-002.patch
        107 kB
        Steve Loughran
      4. HADOOP-13560-branch-2-001.patch
        53 kB
        Steve Loughran

        Issue Links

          Activity

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

          small files of a few tens of MB work; bigger files timeouts start to happen; need to make sure code is resilient to this and reports problems meaningfully

          Show
          stevel@apache.org Steve Loughran added a comment - small files of a few tens of MB work; bigger files timeouts start to happen; need to make sure code is resilient to this and reports problems meaningfully
          Hide
          stevel@apache.org Steve Loughran added a comment -
          1. multi GB files overload heap space when using fast uploader if the mismatch between generation rate and upload throughput is too big.
          2. and you can run out of pooled http connections in the AWS library, as that setting is disconnected
          3. slow uploader doesn't start upload until close(), so takes a lot longer
          4. there's not much in the way of visible metrics as to what is happening until the system finally fails.
          Show
          stevel@apache.org Steve Loughran added a comment - multi GB files overload heap space when using fast uploader if the mismatch between generation rate and upload throughput is too big. and you can run out of pooled http connections in the AWS library, as that setting is disconnected slow uploader doesn't start upload until close(), so takes a lot longer there's not much in the way of visible metrics as to what is happening until the system finally fails.
          Hide
          stevel@apache.org Steve Loughran added a comment - - edited

          Stack trace under load. The client does retry, but it should be blocking further up.

          This doesn't show up with the default settings, as fs.s3a.connection.maximum is at 15, fs.s3a.threads.max at 10. If fs.s3a.threads.max is set to > 15 then things will fail. Retries may complete the work, but it adds delays

          2016-09-01 14:36:06,879 [s3a-transfer-shared-pool1-t5] INFO  http.AmazonHttpClient (AmazonHttpClient.java:executeHelper(496)) - Unable to execute HTTP request: Timeout waiting for connection from pool
          org.apache.http.conn.ConnectionPoolTimeoutException: Timeout waiting for connection from pool
          	at org.apache.http.impl.conn.PoolingClientConnectionManager.leaseConnection(PoolingClientConnectionManager.java:230)
          	at org.apache.http.impl.conn.PoolingClientConnectionManager$1.getConnection(PoolingClientConnectionManager.java:199)
          	at sun.reflect.GeneratedMethodAccessor13.invoke(Unknown Source)
          	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
          	at java.lang.reflect.Method.invoke(Method.java:498)
          	at com.amazonaws.http.conn.ClientConnectionRequestFactory$Handler.invoke(ClientConnectionRequestFactory.java:70)
          	at com.amazonaws.http.conn.$Proxy10.getConnection(Unknown Source)
          	at org.apache.http.impl.client.DefaultRequestDirector.execute(DefaultRequestDirector.java:424)
          	at org.apache.http.impl.client.AbstractHttpClient.doExecute(AbstractHttpClient.java:884)
          	at org.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:82)
          	at org.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:55)
          	at com.amazonaws.http.AmazonHttpClient.executeOneRequest(AmazonHttpClient.java:728)
          	at com.amazonaws.http.AmazonHttpClient.executeHelper(AmazonHttpClient.java:489)
          	at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:310)
          	at com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:3785)
          	at com.amazonaws.services.s3.AmazonS3Client.doUploadPart(AmazonS3Client.java:2921)
          	at com.amazonaws.services.s3.AmazonS3Client.uploadPart(AmazonS3Client.java:2906)
          	at org.apache.hadoop.fs.s3a.S3AFileSystem.uploadPart(S3AFileSystem.java:1080)
          	at org.apache.hadoop.fs.s3a.S3AFastOutputStream$MultiPartUpload$1.call(S3AFastOutputStream.java:358)
          	at org.apache.hadoop.fs.s3a.S3AFastOutputStream$MultiPartUpload$1.call(S3AFastOutputStream.java:353)
          	at org.apache.hadoop.fs.s3a.BlockingThreadPoolExecutorService$CallableWithPermitRelease.call(BlockingThreadPoolExecutorService.java:239)
          	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)
          	at java.lang.Thread.run(Thread.java:745)
          2016-09-01 14:36:06,879 [java-sdk-progress-listener-callback-thread] INFO  scale.STestS3AHugeFileCreate (STestS3AHugeFileCreate.java:progressChanged(221)) - Event CLIENT_REQUEST_RETRY_EVENT, bytes: 0
          ...
          
          Show
          stevel@apache.org Steve Loughran added a comment - - edited Stack trace under load. The client does retry, but it should be blocking further up. This doesn't show up with the default settings, as fs.s3a.connection.maximum is at 15, fs.s3a.threads.max at 10. If fs.s3a.threads.max is set to > 15 then things will fail. Retries may complete the work, but it adds delays 2016-09-01 14:36:06,879 [s3a-transfer-shared-pool1-t5] INFO http.AmazonHttpClient (AmazonHttpClient.java:executeHelper(496)) - Unable to execute HTTP request: Timeout waiting for connection from pool org.apache.http.conn.ConnectionPoolTimeoutException: Timeout waiting for connection from pool at org.apache.http.impl.conn.PoolingClientConnectionManager.leaseConnection(PoolingClientConnectionManager.java:230) at org.apache.http.impl.conn.PoolingClientConnectionManager$1.getConnection(PoolingClientConnectionManager.java:199) at sun.reflect.GeneratedMethodAccessor13.invoke(Unknown Source) at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) at java.lang.reflect.Method.invoke(Method.java:498) at com.amazonaws.http.conn.ClientConnectionRequestFactory$Handler.invoke(ClientConnectionRequestFactory.java:70) at com.amazonaws.http.conn.$Proxy10.getConnection(Unknown Source) at org.apache.http.impl.client.DefaultRequestDirector.execute(DefaultRequestDirector.java:424) at org.apache.http.impl.client.AbstractHttpClient.doExecute(AbstractHttpClient.java:884) at org.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:82) at org.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:55) at com.amazonaws.http.AmazonHttpClient.executeOneRequest(AmazonHttpClient.java:728) at com.amazonaws.http.AmazonHttpClient.executeHelper(AmazonHttpClient.java:489) at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:310) at com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:3785) at com.amazonaws.services.s3.AmazonS3Client.doUploadPart(AmazonS3Client.java:2921) at com.amazonaws.services.s3.AmazonS3Client.uploadPart(AmazonS3Client.java:2906) at org.apache.hadoop.fs.s3a.S3AFileSystem.uploadPart(S3AFileSystem.java:1080) at org.apache.hadoop.fs.s3a.S3AFastOutputStream$MultiPartUpload$1.call(S3AFastOutputStream.java:358) at org.apache.hadoop.fs.s3a.S3AFastOutputStream$MultiPartUpload$1.call(S3AFastOutputStream.java:353) at org.apache.hadoop.fs.s3a.BlockingThreadPoolExecutorService$CallableWithPermitRelease.call(BlockingThreadPoolExecutorService.java:239) 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) at java.lang. Thread .run( Thread .java:745) 2016-09-01 14:36:06,879 [java-sdk-progress-listener-callback-thread] INFO scale.STestS3AHugeFileCreate (STestS3AHugeFileCreate.java:progressChanged(221)) - Event CLIENT_REQUEST_RETRY_EVENT, bytes: 0 ...
          Hide
          stevel@apache.org Steve Loughran added a comment -
          Tests run: 4, Failures: 0, Errors: 1, Skipped: 0, Time elapsed: 434.455 sec <<< FAILURE! - in org.apache.hadoop.fs.s3a.scale.STestS3AHugeFileCreate
          test_010_CreateHugeFile(org.apache.hadoop.fs.s3a.scale.STestS3AHugeFileCreate)  Time elapsed: 336.774 sec  <<< ERROR!
          org.apache.hadoop.fs.s3a.AWSS3IOException: Completing multi-part upload on tests3a/scale/hugefile: com.amazonaws.services.s3.model.AmazonS3Exception: We encountered an internal error. Please try again. (Service: null; Status Code: 0; Error Code: InternalError; Request ID: 15ED37BFBBA92F92), S3 Extended Request ID: +BlJIb5S2QHh1j3dwJysprHTq5iGnvLRD+xWKKld4/0EE3dqt56SwLVZvE4B2100jsN8EKmvVzg=: We encountered an internal error. Please try again. (Service: null; Status Code: 0; Error Code: InternalError; Request ID: 15ED37BFBBA92F92)
          	at com.amazonaws.services.s3.model.transform.XmlResponsesSaxParser$CompleteMultipartUploadHandler.doEndElement(XmlResponsesSaxParser.java:1460)
          	at com.amazonaws.services.s3.model.transform.AbstractHandler.endElement(AbstractHandler.java:52)
          	at com.sun.org.apache.xerces.internal.parsers.AbstractSAXParser.endElement(AbstractSAXParser.java:609)
          	at com.sun.org.apache.xerces.internal.impl.XMLDocumentFragmentScannerImpl.scanEndElement(XMLDocumentFragmentScannerImpl.java:1783)
          	at com.sun.org.apache.xerces.internal.impl.XMLDocumentFragmentScannerImpl$FragmentContentDriver.next(XMLDocumentFragmentScannerImpl.java:2970)
          	at com.sun.org.apache.xerces.internal.impl.XMLDocumentScannerImpl.next(XMLDocumentScannerImpl.java:606)
          	at com.sun.org.apache.xerces.internal.impl.XMLNSDocumentScannerImpl.next(XMLNSDocumentScannerImpl.java:118)
          	at com.sun.org.apache.xerces.internal.impl.XMLDocumentFragmentScannerImpl.scanDocument(XMLDocumentFragmentScannerImpl.java:510)
          	at com.sun.org.apache.xerces.internal.parsers.XML11Configuration.parse(XML11Configuration.java:848)
          	at com.sun.org.apache.xerces.internal.parsers.XML11Configuration.parse(XML11Configuration.java:777)
          	at com.sun.org.apache.xerces.internal.parsers.XMLParser.parse(XMLParser.java:141)
          	at com.sun.org.apache.xerces.internal.parsers.AbstractSAXParser.parse(AbstractSAXParser.java:1213)
          	at com.amazonaws.services.s3.model.transform.XmlResponsesSaxParser.parseXmlInputStream(XmlResponsesSaxParser.java:151)
          	at com.amazonaws.services.s3.model.transform.XmlResponsesSaxParser.parseCompleteMultipartUploadResponse(XmlResponsesSaxParser.java:444)
          	at com.amazonaws.services.s3.model.transform.Unmarshallers$CompleteMultipartUploadResultUnmarshaller.unmarshall(Unmarshallers.java:213)
          	at com.amazonaws.services.s3.model.transform.Unmarshallers$CompleteMultipartUploadResultUnmarshaller.unmarshall(Unmarshallers.java:210)
          	at com.amazonaws.services.s3.internal.S3XmlResponseHandler.handle(S3XmlResponseHandler.java:62)
          	at com.amazonaws.services.s3.internal.ResponseHeaderHandlerChain.handle(ResponseHeaderHandlerChain.java:44)
          	at com.amazonaws.services.s3.internal.ResponseHeaderHandlerChain.handle(ResponseHeaderHandlerChain.java:30)
          	at com.amazonaws.http.AmazonHttpClient.handleResponse(AmazonHttpClient.java:1072)
          	at com.amazonaws.http.AmazonHttpClient.executeOneRequest(AmazonHttpClient.java:746)
          	at com.amazonaws.http.AmazonHttpClient.executeHelper(AmazonHttpClient.java:489)
          	at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:310)
          	at com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:3785)
          	at com.amazonaws.services.s3.AmazonS3Client.completeMultipartUpload(AmazonS3Client.java:2705)
          	at org.apache.hadoop.fs.s3a.S3AFastOutputStream$MultiPartUpload.complete(S3AFastOutputStream.java:388)
          	at org.apache.hadoop.fs.s3a.S3AFastOutputStream$MultiPartUpload.access$200(S3AFastOutputStream.java:333)
          	at org.apache.hadoop.fs.s3a.S3AFastOutputStream.close(S3AFastOutputStream.java:270)
          	at org.apache.hadoop.fs.FSDataOutputStream$PositionCache.close(FSDataOutputStream.java:72)
          	at org.apache.hadoop.fs.FSDataOutputStream.close(FSDataOutputStream.java:106)
          	at org.apache.hadoop.fs.s3a.scale.STestS3AHugeFileCreate.test_010_CreateHugeFile(STestS3AHugeFileCreate.java:161)
          
          Show
          stevel@apache.org Steve Loughran added a comment - Tests run: 4, Failures: 0, Errors: 1, Skipped: 0, Time elapsed: 434.455 sec <<< FAILURE! - in org.apache.hadoop.fs.s3a.scale.STestS3AHugeFileCreate test_010_CreateHugeFile(org.apache.hadoop.fs.s3a.scale.STestS3AHugeFileCreate) Time elapsed: 336.774 sec <<< ERROR! org.apache.hadoop.fs.s3a.AWSS3IOException: Completing multi-part upload on tests3a/scale/hugefile: com.amazonaws.services.s3.model.AmazonS3Exception: We encountered an internal error. Please try again. (Service: null ; Status Code: 0; Error Code: InternalError; Request ID: 15ED37BFBBA92F92), S3 Extended Request ID: +BlJIb5S2QHh1j3dwJysprHTq5iGnvLRD+xWKKld4/0EE3dqt56SwLVZvE4B2100jsN8EKmvVzg=: We encountered an internal error. Please try again. (Service: null ; Status Code: 0; Error Code: InternalError; Request ID: 15ED37BFBBA92F92) at com.amazonaws.services.s3.model.transform.XmlResponsesSaxParser$CompleteMultipartUploadHandler.doEndElement(XmlResponsesSaxParser.java:1460) at com.amazonaws.services.s3.model.transform.AbstractHandler.endElement(AbstractHandler.java:52) at com.sun.org.apache.xerces.internal.parsers.AbstractSAXParser.endElement(AbstractSAXParser.java:609) at com.sun.org.apache.xerces.internal.impl.XMLDocumentFragmentScannerImpl.scanEndElement(XMLDocumentFragmentScannerImpl.java:1783) at com.sun.org.apache.xerces.internal.impl.XMLDocumentFragmentScannerImpl$FragmentContentDriver.next(XMLDocumentFragmentScannerImpl.java:2970) at com.sun.org.apache.xerces.internal.impl.XMLDocumentScannerImpl.next(XMLDocumentScannerImpl.java:606) at com.sun.org.apache.xerces.internal.impl.XMLNSDocumentScannerImpl.next(XMLNSDocumentScannerImpl.java:118) at com.sun.org.apache.xerces.internal.impl.XMLDocumentFragmentScannerImpl.scanDocument(XMLDocumentFragmentScannerImpl.java:510) at com.sun.org.apache.xerces.internal.parsers.XML11Configuration.parse(XML11Configuration.java:848) at com.sun.org.apache.xerces.internal.parsers.XML11Configuration.parse(XML11Configuration.java:777) at com.sun.org.apache.xerces.internal.parsers.XMLParser.parse(XMLParser.java:141) at com.sun.org.apache.xerces.internal.parsers.AbstractSAXParser.parse(AbstractSAXParser.java:1213) at com.amazonaws.services.s3.model.transform.XmlResponsesSaxParser.parseXmlInputStream(XmlResponsesSaxParser.java:151) at com.amazonaws.services.s3.model.transform.XmlResponsesSaxParser.parseCompleteMultipartUploadResponse(XmlResponsesSaxParser.java:444) at com.amazonaws.services.s3.model.transform.Unmarshallers$CompleteMultipartUploadResultUnmarshaller.unmarshall(Unmarshallers.java:213) at com.amazonaws.services.s3.model.transform.Unmarshallers$CompleteMultipartUploadResultUnmarshaller.unmarshall(Unmarshallers.java:210) at com.amazonaws.services.s3.internal.S3XmlResponseHandler.handle(S3XmlResponseHandler.java:62) at com.amazonaws.services.s3.internal.ResponseHeaderHandlerChain.handle(ResponseHeaderHandlerChain.java:44) at com.amazonaws.services.s3.internal.ResponseHeaderHandlerChain.handle(ResponseHeaderHandlerChain.java:30) at com.amazonaws.http.AmazonHttpClient.handleResponse(AmazonHttpClient.java:1072) at com.amazonaws.http.AmazonHttpClient.executeOneRequest(AmazonHttpClient.java:746) at com.amazonaws.http.AmazonHttpClient.executeHelper(AmazonHttpClient.java:489) at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:310) at com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:3785) at com.amazonaws.services.s3.AmazonS3Client.completeMultipartUpload(AmazonS3Client.java:2705) at org.apache.hadoop.fs.s3a.S3AFastOutputStream$MultiPartUpload.complete(S3AFastOutputStream.java:388) at org.apache.hadoop.fs.s3a.S3AFastOutputStream$MultiPartUpload.access$200(S3AFastOutputStream.java:333) at org.apache.hadoop.fs.s3a.S3AFastOutputStream.close(S3AFastOutputStream.java:270) at org.apache.hadoop.fs.FSDataOutputStream$PositionCache.close(FSDataOutputStream.java:72) at org.apache.hadoop.fs.FSDataOutputStream.close(FSDataOutputStream.java:106) at org.apache.hadoop.fs.s3a.scale.STestS3AHugeFileCreate.test_010_CreateHugeFile(STestS3AHugeFileCreate.java:161)
          Hide
          stevel@apache.org Steve Loughran added a comment -
          Tests run: 4, Failures: 0, Errors: 1, Skipped: 0, Time elapsed: 434.455 sec <<< FAILURE! - in org.apache.hadoop.fs.s3a.scale.STestS3AHugeFileCreate
          test_010_CreateHugeFile(org.apache.hadoop.fs.s3a.scale.STestS3AHugeFileCreate)  Time elapsed: 336.774 sec  <<< ERROR!
          org.apache.hadoop.fs.s3a.AWSS3IOException: Completing multi-part upload on tests3a/scale/hugefile: com.amazonaws.services.s3.model.AmazonS3Exception: We encountered an internal error. Please try again. (Service: null; Status Code: 0; Error Code: InternalError; Request ID: 15ED37BFBBA92F92), S3 Extended Request ID: +BlJIb5S2QHh1j3dwJysprHTq5iGnvLRD+xWKKld4/0EE3dqt56SwLVZvE4B2100jsN8EKmvVzg=: We encountered an internal error. Please try again. (Service: null; Status Code: 0; Error Code: InternalError; Request ID: 15ED37BFBBA92F92)
          	at com.amazonaws.services.s3.model.transform.XmlResponsesSaxParser$CompleteMultipartUploadHandler.doEndElement(XmlResponsesSaxParser.java:1460)
          	at com.amazonaws.services.s3.model.transform.AbstractHandler.endElement(AbstractHandler.java:52)
          	at com.sun.org.apache.xerces.internal.parsers.AbstractSAXParser.endElement(AbstractSAXParser.java:609)
          	at com.sun.org.apache.xerces.internal.impl.XMLDocumentFragmentScannerImpl.scanEndElement(XMLDocumentFragmentScannerImpl.java:1783)
          	at com.sun.org.apache.xerces.internal.impl.XMLDocumentFragmentScannerImpl$FragmentContentDriver.next(XMLDocumentFragmentScannerImpl.java:2970)
          	at com.sun.org.apache.xerces.internal.impl.XMLDocumentScannerImpl.next(XMLDocumentScannerImpl.java:606)
          	at com.sun.org.apache.xerces.internal.impl.XMLNSDocumentScannerImpl.next(XMLNSDocumentScannerImpl.java:118)
          	at com.sun.org.apache.xerces.internal.impl.XMLDocumentFragmentScannerImpl.scanDocument(XMLDocumentFragmentScannerImpl.java:510)
          	at com.sun.org.apache.xerces.internal.parsers.XML11Configuration.parse(XML11Configuration.java:848)
          	at com.sun.org.apache.xerces.internal.parsers.XML11Configuration.parse(XML11Configuration.java:777)
          	at com.sun.org.apache.xerces.internal.parsers.XMLParser.parse(XMLParser.java:141)
          	at com.sun.org.apache.xerces.internal.parsers.AbstractSAXParser.parse(AbstractSAXParser.java:1213)
          	at com.amazonaws.services.s3.model.transform.XmlResponsesSaxParser.parseXmlInputStream(XmlResponsesSaxParser.java:151)
          	at com.amazonaws.services.s3.model.transform.XmlResponsesSaxParser.parseCompleteMultipartUploadResponse(XmlResponsesSaxParser.java:444)
          	at com.amazonaws.services.s3.model.transform.Unmarshallers$CompleteMultipartUploadResultUnmarshaller.unmarshall(Unmarshallers.java:213)
          	at com.amazonaws.services.s3.model.transform.Unmarshallers$CompleteMultipartUploadResultUnmarshaller.unmarshall(Unmarshallers.java:210)
          	at com.amazonaws.services.s3.internal.S3XmlResponseHandler.handle(S3XmlResponseHandler.java:62)
          	at com.amazonaws.services.s3.internal.ResponseHeaderHandlerChain.handle(ResponseHeaderHandlerChain.java:44)
          	at com.amazonaws.services.s3.internal.ResponseHeaderHandlerChain.handle(ResponseHeaderHandlerChain.java:30)
          	at com.amazonaws.http.AmazonHttpClient.handleResponse(AmazonHttpClient.java:1072)
          	at com.amazonaws.http.AmazonHttpClient.executeOneRequest(AmazonHttpClient.java:746)
          	at com.amazonaws.http.AmazonHttpClient.executeHelper(AmazonHttpClient.java:489)
          	at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:310)
          	at com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:3785)
          	at com.amazonaws.services.s3.AmazonS3Client.completeMultipartUpload(AmazonS3Client.java:2705)
          	at org.apache.hadoop.fs.s3a.S3AFastOutputStream$MultiPartUpload.complete(S3AFastOutputStream.java:388)
          	at org.apache.hadoop.fs.s3a.S3AFastOutputStream$MultiPartUpload.access$200(S3AFastOutputStream.java:333)
          	at org.apache.hadoop.fs.s3a.S3AFastOutputStream.close(S3AFastOutputStream.java:270)
          	at org.apache.hadoop.fs.FSDataOutputStream$PositionCache.close(FSDataOutputStream.java:72)
          	at org.apache.hadoop.fs.FSDataOutputStream.close(FSDataOutputStream.java:106)
          	at org.apache.hadoop.fs.s3a.scale.STestS3AHugeFileCreate.test_010_CreateHugeFile(STestS3AHugeFileCreate.java:161)
          
          Show
          stevel@apache.org Steve Loughran added a comment - Tests run: 4, Failures: 0, Errors: 1, Skipped: 0, Time elapsed: 434.455 sec <<< FAILURE! - in org.apache.hadoop.fs.s3a.scale.STestS3AHugeFileCreate test_010_CreateHugeFile(org.apache.hadoop.fs.s3a.scale.STestS3AHugeFileCreate) Time elapsed: 336.774 sec <<< ERROR! org.apache.hadoop.fs.s3a.AWSS3IOException: Completing multi-part upload on tests3a/scale/hugefile: com.amazonaws.services.s3.model.AmazonS3Exception: We encountered an internal error. Please try again. (Service: null ; Status Code: 0; Error Code: InternalError; Request ID: 15ED37BFBBA92F92), S3 Extended Request ID: +BlJIb5S2QHh1j3dwJysprHTq5iGnvLRD+xWKKld4/0EE3dqt56SwLVZvE4B2100jsN8EKmvVzg=: We encountered an internal error. Please try again. (Service: null ; Status Code: 0; Error Code: InternalError; Request ID: 15ED37BFBBA92F92) at com.amazonaws.services.s3.model.transform.XmlResponsesSaxParser$CompleteMultipartUploadHandler.doEndElement(XmlResponsesSaxParser.java:1460) at com.amazonaws.services.s3.model.transform.AbstractHandler.endElement(AbstractHandler.java:52) at com.sun.org.apache.xerces.internal.parsers.AbstractSAXParser.endElement(AbstractSAXParser.java:609) at com.sun.org.apache.xerces.internal.impl.XMLDocumentFragmentScannerImpl.scanEndElement(XMLDocumentFragmentScannerImpl.java:1783) at com.sun.org.apache.xerces.internal.impl.XMLDocumentFragmentScannerImpl$FragmentContentDriver.next(XMLDocumentFragmentScannerImpl.java:2970) at com.sun.org.apache.xerces.internal.impl.XMLDocumentScannerImpl.next(XMLDocumentScannerImpl.java:606) at com.sun.org.apache.xerces.internal.impl.XMLNSDocumentScannerImpl.next(XMLNSDocumentScannerImpl.java:118) at com.sun.org.apache.xerces.internal.impl.XMLDocumentFragmentScannerImpl.scanDocument(XMLDocumentFragmentScannerImpl.java:510) at com.sun.org.apache.xerces.internal.parsers.XML11Configuration.parse(XML11Configuration.java:848) at com.sun.org.apache.xerces.internal.parsers.XML11Configuration.parse(XML11Configuration.java:777) at com.sun.org.apache.xerces.internal.parsers.XMLParser.parse(XMLParser.java:141) at com.sun.org.apache.xerces.internal.parsers.AbstractSAXParser.parse(AbstractSAXParser.java:1213) at com.amazonaws.services.s3.model.transform.XmlResponsesSaxParser.parseXmlInputStream(XmlResponsesSaxParser.java:151) at com.amazonaws.services.s3.model.transform.XmlResponsesSaxParser.parseCompleteMultipartUploadResponse(XmlResponsesSaxParser.java:444) at com.amazonaws.services.s3.model.transform.Unmarshallers$CompleteMultipartUploadResultUnmarshaller.unmarshall(Unmarshallers.java:213) at com.amazonaws.services.s3.model.transform.Unmarshallers$CompleteMultipartUploadResultUnmarshaller.unmarshall(Unmarshallers.java:210) at com.amazonaws.services.s3.internal.S3XmlResponseHandler.handle(S3XmlResponseHandler.java:62) at com.amazonaws.services.s3.internal.ResponseHeaderHandlerChain.handle(ResponseHeaderHandlerChain.java:44) at com.amazonaws.services.s3.internal.ResponseHeaderHandlerChain.handle(ResponseHeaderHandlerChain.java:30) at com.amazonaws.http.AmazonHttpClient.handleResponse(AmazonHttpClient.java:1072) at com.amazonaws.http.AmazonHttpClient.executeOneRequest(AmazonHttpClient.java:746) at com.amazonaws.http.AmazonHttpClient.executeHelper(AmazonHttpClient.java:489) at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:310) at com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:3785) at com.amazonaws.services.s3.AmazonS3Client.completeMultipartUpload(AmazonS3Client.java:2705) at org.apache.hadoop.fs.s3a.S3AFastOutputStream$MultiPartUpload.complete(S3AFastOutputStream.java:388) at org.apache.hadoop.fs.s3a.S3AFastOutputStream$MultiPartUpload.access$200(S3AFastOutputStream.java:333) at org.apache.hadoop.fs.s3a.S3AFastOutputStream.close(S3AFastOutputStream.java:270) at org.apache.hadoop.fs.FSDataOutputStream$PositionCache.close(FSDataOutputStream.java:72) at org.apache.hadoop.fs.FSDataOutputStream.close(FSDataOutputStream.java:106) at org.apache.hadoop.fs.s3a.scale.STestS3AHugeFileCreate.test_010_CreateHugeFile(STestS3AHugeFileCreate.java:161)
          Hide
          stevel@apache.org Steve Loughran added a comment -

          This is a (potentially transient) failure trying to commit the operation. This could be pretty dramatic when trying to commit a large file: better to have a retry policy. Going for a hard-coded retryUpToMaximumCountWithProportionalSleep policy initially. We may need to determine what if any error code there is for unknown/already completed upload, and not retry on those, instead treatng them as a sign the operation previously completed.

          Also: do the same for abort()

          Show
          stevel@apache.org Steve Loughran added a comment - This is a (potentially transient) failure trying to commit the operation. This could be pretty dramatic when trying to commit a large file: better to have a retry policy. Going for a hard-coded retryUpToMaximumCountWithProportionalSleep policy initially. We may need to determine what if any error code there is for unknown/already completed upload, and not retry on those, instead treatng them as a sign the operation previously completed. Also: do the same for abort()
          Hide
          stevel@apache.org Steve Loughran added a comment -

          Patch 001

          1. scale test which can create a configurable file; good to try with small (<1 part) uploads as well as medium, and uploads so big they bring your heap down.
          2. New Scale Test (prefix STest) likes lots of upload bandwidth. It's running non-parallelized...I wonder if it should be left out of integration entirely.
          3. they are also good for demonstrating the failure modes of S3A bits
          4. Note how test methods explicitly ask JUnit to run in a given order. Allows the tests to isolate their operations yet still work in sequence.
          5. The upload one does a no-op if the destination file of that size exists. It was meant to let you skip that bit if it was already there...I think I'll pull that feature as it only gets in the way.
            $ lots of extra monitoring of what is going on inside S3A, including a gauge of active PUT request counts and bytes pending.
          6. more troubleshooting docs.
          7. The fast output stream will retry on errors during request completion and abort.
          • HADOOP-13569 S3AFastOutputStream to take ProgressListener in file create(); used in test runs
          • HADOOP-13566 NPE in S3AFastOutputStream.write
          • HADOOP-13567 S3AFileSystem to override getStoragetStatistics() and so serve up its statistics
          Show
          stevel@apache.org Steve Loughran added a comment - Patch 001 scale test which can create a configurable file; good to try with small (<1 part) uploads as well as medium, and uploads so big they bring your heap down. New Scale Test (prefix STest) likes lots of upload bandwidth. It's running non-parallelized...I wonder if it should be left out of integration entirely. they are also good for demonstrating the failure modes of S3A bits Note how test methods explicitly ask JUnit to run in a given order. Allows the tests to isolate their operations yet still work in sequence. The upload one does a no-op if the destination file of that size exists. It was meant to let you skip that bit if it was already there...I think I'll pull that feature as it only gets in the way. $ lots of extra monitoring of what is going on inside S3A, including a gauge of active PUT request counts and bytes pending. more troubleshooting docs. The fast output stream will retry on errors during request completion and abort. HADOOP-13569 S3AFastOutputStream to take ProgressListener in file create(); used in test runs HADOOP-13566 NPE in S3AFastOutputStream.write HADOOP-13567 S3AFileSystem to override getStoragetStatistics() and so serve up its statistics
          Hide
          stevel@apache.org Steve Loughran added a comment -

          Tested: s3a ireland

          Show
          stevel@apache.org Steve Loughran added a comment - Tested: s3a ireland
          Hide
          hadoopqa Hadoop QA added a comment -
          -1 overall



          Vote Subsystem Runtime Comment
          0 reexec 0m 24s Docker mode activated.
          +1 @author 0m 0s The patch does not contain any @author tags.
          +1 test4tests 0m 0s The patch appears to include 8 new or modified test files.
          +1 mvninstall 7m 9s branch-2 passed
          +1 compile 0m 15s branch-2 passed with JDK v1.8.0_101
          +1 compile 0m 19s branch-2 passed with JDK v1.7.0_111
          +1 checkstyle 0m 15s branch-2 passed
          +1 mvnsite 0m 24s branch-2 passed
          +1 mvneclipse 1m 3s branch-2 passed
          +1 findbugs 0m 34s branch-2 passed
          +1 javadoc 0m 13s branch-2 passed with JDK v1.8.0_101
          +1 javadoc 0m 15s branch-2 passed with JDK v1.7.0_111
          +1 mvninstall 0m 18s the patch passed
          +1 compile 0m 13s the patch passed with JDK v1.8.0_101
          +1 javac 0m 13s the patch passed
          +1 compile 0m 16s the patch passed with JDK v1.7.0_111
          +1 javac 0m 16s the patch passed
          -0 checkstyle 0m 12s hadoop-tools/hadoop-aws: The patch generated 21 new + 30 unchanged - 0 fixed = 51 total (was 30)
          +1 mvnsite 0m 21s the patch passed
          +1 mvneclipse 0m 12s the patch passed
          -1 whitespace 0m 0s The patch has 3 line(s) that end in whitespace. Use git apply --whitespace=fix <<patch_file>>. Refer https://git-scm.com/docs/git-apply
          -1 whitespace 0m 0s The patch 25 line(s) with tabs.
          +1 xml 0m 0s The patch has no ill-formed XML file.
          +1 findbugs 0m 42s the patch passed
          +1 javadoc 0m 11s the patch passed with JDK v1.8.0_101
          +1 javadoc 0m 13s the patch passed with JDK v1.7.0_111
          +1 unit 0m 19s hadoop-aws in the patch passed with JDK v1.7.0_111.
          +1 asflicense 0m 17s The patch does not generate ASF License warnings.
          16m 14s



          Subsystem Report/Notes
          Docker Image:yetus/hadoop:b59b8b7
          JIRA Issue HADOOP-13560
          JIRA Patch URL https://issues.apache.org/jira/secure/attachment/12826665/HADOOP-13560-branch-2-001.patch
          Optional Tests asflicense compile javac javadoc mvninstall mvnsite unit xml findbugs checkstyle
          uname Linux dab73e3f6727 3.13.0-36-lowlatency #63-Ubuntu SMP PREEMPT Wed Sep 3 21:56:12 UTC 2014 x86_64 x86_64 x86_64 GNU/Linux
          Build tool maven
          Personality /testptch/hadoop/patchprocess/precommit/personality/provided.sh
          git revision branch-2 / 34f9330
          Default Java 1.7.0_111
          Multi-JDK versions /usr/lib/jvm/java-8-oracle:1.8.0_101 /usr/lib/jvm/java-7-openjdk-amd64:1.7.0_111
          findbugs v3.0.0
          checkstyle https://builds.apache.org/job/PreCommit-HADOOP-Build/10439/artifact/patchprocess/diff-checkstyle-hadoop-tools_hadoop-aws.txt
          whitespace https://builds.apache.org/job/PreCommit-HADOOP-Build/10439/artifact/patchprocess/whitespace-eol.txt
          whitespace https://builds.apache.org/job/PreCommit-HADOOP-Build/10439/artifact/patchprocess/whitespace-tabs.txt
          JDK v1.7.0_111 Test Results https://builds.apache.org/job/PreCommit-HADOOP-Build/10439/testReport/
          modules C: hadoop-tools/hadoop-aws U: hadoop-tools/hadoop-aws
          Console output https://builds.apache.org/job/PreCommit-HADOOP-Build/10439/console
          Powered by Apache Yetus 0.4.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 24s Docker mode activated. +1 @author 0m 0s The patch does not contain any @author tags. +1 test4tests 0m 0s The patch appears to include 8 new or modified test files. +1 mvninstall 7m 9s branch-2 passed +1 compile 0m 15s branch-2 passed with JDK v1.8.0_101 +1 compile 0m 19s branch-2 passed with JDK v1.7.0_111 +1 checkstyle 0m 15s branch-2 passed +1 mvnsite 0m 24s branch-2 passed +1 mvneclipse 1m 3s branch-2 passed +1 findbugs 0m 34s branch-2 passed +1 javadoc 0m 13s branch-2 passed with JDK v1.8.0_101 +1 javadoc 0m 15s branch-2 passed with JDK v1.7.0_111 +1 mvninstall 0m 18s the patch passed +1 compile 0m 13s the patch passed with JDK v1.8.0_101 +1 javac 0m 13s the patch passed +1 compile 0m 16s the patch passed with JDK v1.7.0_111 +1 javac 0m 16s the patch passed -0 checkstyle 0m 12s hadoop-tools/hadoop-aws: The patch generated 21 new + 30 unchanged - 0 fixed = 51 total (was 30) +1 mvnsite 0m 21s the patch passed +1 mvneclipse 0m 12s the patch passed -1 whitespace 0m 0s The patch has 3 line(s) that end in whitespace. Use git apply --whitespace=fix <<patch_file>>. Refer https://git-scm.com/docs/git-apply -1 whitespace 0m 0s The patch 25 line(s) with tabs. +1 xml 0m 0s The patch has no ill-formed XML file. +1 findbugs 0m 42s the patch passed +1 javadoc 0m 11s the patch passed with JDK v1.8.0_101 +1 javadoc 0m 13s the patch passed with JDK v1.7.0_111 +1 unit 0m 19s hadoop-aws in the patch passed with JDK v1.7.0_111. +1 asflicense 0m 17s The patch does not generate ASF License warnings. 16m 14s Subsystem Report/Notes Docker Image:yetus/hadoop:b59b8b7 JIRA Issue HADOOP-13560 JIRA Patch URL https://issues.apache.org/jira/secure/attachment/12826665/HADOOP-13560-branch-2-001.patch Optional Tests asflicense compile javac javadoc mvninstall mvnsite unit xml findbugs checkstyle uname Linux dab73e3f6727 3.13.0-36-lowlatency #63-Ubuntu SMP PREEMPT Wed Sep 3 21:56:12 UTC 2014 x86_64 x86_64 x86_64 GNU/Linux Build tool maven Personality /testptch/hadoop/patchprocess/precommit/personality/provided.sh git revision branch-2 / 34f9330 Default Java 1.7.0_111 Multi-JDK versions /usr/lib/jvm/java-8-oracle:1.8.0_101 /usr/lib/jvm/java-7-openjdk-amd64:1.7.0_111 findbugs v3.0.0 checkstyle https://builds.apache.org/job/PreCommit-HADOOP-Build/10439/artifact/patchprocess/diff-checkstyle-hadoop-tools_hadoop-aws.txt whitespace https://builds.apache.org/job/PreCommit-HADOOP-Build/10439/artifact/patchprocess/whitespace-eol.txt whitespace https://builds.apache.org/job/PreCommit-HADOOP-Build/10439/artifact/patchprocess/whitespace-tabs.txt JDK v1.7.0_111 Test Results https://builds.apache.org/job/PreCommit-HADOOP-Build/10439/testReport/ modules C: hadoop-tools/hadoop-aws U: hadoop-tools/hadoop-aws Console output https://builds.apache.org/job/PreCommit-HADOOP-Build/10439/console Powered by Apache Yetus 0.4.0-SNAPSHOT http://yetus.apache.org This message was automatically generated.
          Hide
          stevel@apache.org Steve Loughran added a comment -

          Cancelling patch to do the fix for this

          1. Support multiple back ends for buffering streams (current one for regression testing, java nio bytebuffer for real use & potentially off-heap storage, and files fixed to partition size. The latter will have no limit on size other than HDD capacity & do async upload for incremental writes, rather than the big delay on close(). This will mean that it should be usable everywhere.
          2. collect metrics on upload performance: #of writes, number active, delay between submitting to the executor pool and execution, per-stream upload bandwidth as inferred from duration of the upload operations, per-stream and per FS instance. That will let you measure what is going on w.r.t s3a upload performance in production, including collecting stats of what b/w you get from different VM instances.
          Show
          stevel@apache.org Steve Loughran added a comment - Cancelling patch to do the fix for this Support multiple back ends for buffering streams (current one for regression testing, java nio bytebuffer for real use & potentially off-heap storage, and files fixed to partition size. The latter will have no limit on size other than HDD capacity & do async upload for incremental writes, rather than the big delay on close(). This will mean that it should be usable everywhere. collect metrics on upload performance: #of writes, number active, delay between submitting to the executor pool and execution, per-stream upload bandwidth as inferred from duration of the upload operations, per-stream and per FS instance. That will let you measure what is going on w.r.t s3a upload performance in production, including collecting stats of what b/w you get from different VM instances.
          Hide
          stevel@apache.org Steve Loughran added a comment -

          HADOOP-13560 patch 002
          block streaming is in, testing at moderate scale <100 MB.

          you can choose for buffer-by-ram (current fast uploader) or buffer by HDD; in a test using SSD & remote S3, I got ~1.38MB/s bandwidth, got something similar 1.44 on RAM. But: we shouldn't run out off heap on the HDD option. RAM buffering uses existing ByteArrays, to ease source code migration off FastUpload (which is still there, for now).

          • Next: the multi GB tests
          • I do plan to add pooled ByteBuffers
          • Add metrics of total and ongoing upload, including tracking what quantity of the outstanding block data has actually been uploaded.
          Show
          stevel@apache.org Steve Loughran added a comment - HADOOP-13560 patch 002 block streaming is in, testing at moderate scale <100 MB. you can choose for buffer-by-ram (current fast uploader) or buffer by HDD; in a test using SSD & remote S3, I got ~1.38MB/s bandwidth, got something similar 1.44 on RAM. But: we shouldn't run out off heap on the HDD option. RAM buffering uses existing ByteArrays, to ease source code migration off FastUpload (which is still there, for now). Next: the multi GB tests I do plan to add pooled ByteBuffers Add metrics of total and ongoing upload, including tracking what quantity of the outstanding block data has actually been uploaded.
          Hide
          stevel@apache.org Steve Loughran added a comment -

          testing ongoing against S3 ireland

          Show
          stevel@apache.org Steve Loughran added a comment - testing ongoing against S3 ireland
          Hide
          hadoopqa Hadoop QA added a comment -
          -1 overall



          Vote Subsystem Runtime Comment
          0 reexec 0m 0s Docker mode activated.
          -1 patch 0m 10s HADOOP-13560 does not apply to branch-2. Rebase required? Wrong Branch? See https://wiki.apache.org/hadoop/HowToContribute for help.



          Subsystem Report/Notes
          JIRA Issue HADOOP-13560
          JIRA Patch URL https://issues.apache.org/jira/secure/attachment/12827245/HADOOP-13560-branch-2-002.patch
          Console output https://builds.apache.org/job/PreCommit-HADOOP-Build/10453/console
          Powered by Apache Yetus 0.4.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 0s Docker mode activated. -1 patch 0m 10s HADOOP-13560 does not apply to branch-2. Rebase required? Wrong Branch? See https://wiki.apache.org/hadoop/HowToContribute for help. Subsystem Report/Notes JIRA Issue HADOOP-13560 JIRA Patch URL https://issues.apache.org/jira/secure/attachment/12827245/HADOOP-13560-branch-2-002.patch Console output https://builds.apache.org/job/PreCommit-HADOOP-Build/10453/console Powered by Apache Yetus 0.4.0-SNAPSHOT http://yetus.apache.org This message was automatically generated.
          Hide
          stevel@apache.org Steve Loughran added a comment -

          2GB uploads when file buffered fails; AWS complains

          Running org.apache.hadoop.fs.s3a.scale.STestS3AHugeFilesDiskBlocks
          Tests run: 5, Failures: 0, Errors: 1, Skipped: 3, Time elapsed: 1,258.424 sec <<< FAILURE! - in org.apache.hadoop.fs.s3a.scale.STestS3AHugeFilesDiskBlocks
          test_010_CreateHugeFile(org.apache.hadoop.fs.s3a.scale.STestS3AHugeFilesDiskBlocks)  Time elapsed: 1,256.013 sec  <<< ERROR!
          org.apache.hadoop.fs.s3a.AWSClientIOException: Multi-part upload with id 'dZga.hig99Nxdm1S5dlcilzpg1kiav7ZF2QCJZZydN0qyE7U_pMUEYdACOavY_us3q9CgIxfKaQadXLhgUseUw--' on tests3a/scale/hugefile: com.amazonaws.ResetException: Failed to reset the request input stream;  If the request involves an input stream, the maximum stream buffer size can be configured via request.getRequestClientOptions().setReadLimit(int): Failed to reset the request input stream;  If the request involves an input stream, the maximum stream buffer size can be configured via request.getRequestClientOptions().setReadLimit(int)
          	at org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:108)
          	at org.apache.hadoop.fs.s3a.S3AUtils.extractException(S3AUtils.java:165)
          	at org.apache.hadoop.fs.s3a.S3ABlockOutputStream$MultiPartUpload.waitForAllPartUploads(S3ABlockOutputStream.java:418)
          	at org.apache.hadoop.fs.s3a.S3ABlockOutputStream$MultiPartUpload.access$100(S3ABlockOutputStream.java:356)
          	at org.apache.hadoop.fs.s3a.S3ABlockOutputStream.close(S3ABlockOutputStream.java:261)
          	at org.apache.hadoop.fs.FSDataOutputStream$PositionCache.close(FSDataOutputStream.java:72)
          	at org.apache.hadoop.fs.FSDataOutputStream.close(FSDataOutputStream.java:106)
          	at org.apache.hadoop.fs.s3a.scale.AbstractSTestS3AHugeFiles.test_010_CreateHugeFile(AbstractSTestS3AHugeFiles.java:149)
          	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: com.amazonaws.ResetException: Failed to reset the request input stream;  If the request involves an input stream, the maximum stream buffer size can be configured via request.getRequestClientOptions().setReadLimit(int)
          	at com.amazonaws.http.AmazonHttpClient.executeOneRequest(AmazonHttpClient.java:665)
          	at com.amazonaws.http.AmazonHttpClient.executeHelper(AmazonHttpClient.java:489)
          	at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:310)
          	at com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:3785)
          	at com.amazonaws.services.s3.AmazonS3Client.doUploadPart(AmazonS3Client.java:2921)
          	at com.amazonaws.services.s3.AmazonS3Client.uploadPart(AmazonS3Client.java:2906)
          	at org.apache.hadoop.fs.s3a.S3AFileSystem.uploadPart(S3AFileSystem.java:1141)
          	at org.apache.hadoop.fs.s3a.S3ABlockOutputStream$MultiPartUpload$1.call(S3ABlockOutputStream.java:391)
          	at org.apache.hadoop.fs.s3a.S3ABlockOutputStream$MultiPartUpload$1.call(S3ABlockOutputStream.java:384)
          	at org.apache.hadoop.fs.s3a.BlockingThreadPoolExecutorService$CallableWithPermitRelease.call(BlockingThreadPoolExecutorService.java:239)
          	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)
          	at java.lang.Thread.run(Thread.java:745)
          Caused by: java.io.IOException: Resetting to invalid mark
          	at java.io.BufferedInputStream.reset(BufferedInputStream.java:448)
          	at org.apache.hadoop.fs.s3a.S3ADataBlocks$ForwardingInputStream.reset(S3ADataBlocks.java:432)
          	at com.amazonaws.internal.SdkFilterInputStream.reset(SdkFilterInputStream.java:102)
          	at com.amazonaws.internal.SdkFilterInputStream.reset(SdkFilterInputStream.java:102)
          	at com.amazonaws.services.s3.internal.InputSubstream.reset(InputSubstream.java:110)
          	at com.amazonaws.internal.SdkFilterInputStream.reset(SdkFilterInputStream.java:102)
          	at com.amazonaws.services.s3.internal.MD5DigestCalculatingInputStream.reset(MD5DigestCalculatingInputStream.java:76)
          	at com.amazonaws.internal.SdkFilterInputStream.reset(SdkFilterInputStream.java:102)
          	at com.amazonaws.event.ProgressInputStream.reset(ProgressInputStream.java:139)
          	at com.amazonaws.internal.SdkFilterInputStream.reset(SdkFilterInputStream.java:102)
          	at com.amazonaws.http.AmazonHttpClient.executeOneRequest(AmazonHttpClient.java:663)
          	at com.amazonaws.http.AmazonHttpClient.executeHelper(AmazonHttpClient.java:489)
          	at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:310)
          	at com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:3785)
          	at com.amazonaws.services.s3.AmazonS3Client.doUploadPart(AmazonS3Client.java:2921)
          	at com.amazonaws.services.s3.AmazonS3Client.uploadPart(AmazonS3Client.java:2906)
          	at org.apache.hadoop.fs.s3a.S3AFileSystem.uploadPart(S3AFileSystem.java:1141)
          	at org.apache.hadoop.fs.s3a.S3ABlockOutputStream$MultiPartUpload$1.call(S3ABlockOutputStream.java:391)
          	at org.apache.hadoop.fs.s3a.S3ABlockOutputStream$MultiPartUpload$1.call(S3ABlockOutputStream.java:384)
          	at org.apache.hadoop.fs.s3a.BlockingThreadPoolExecutorService$CallableWithPermitRelease.call(BlockingThreadPoolExecutorService.java:239)
          	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)
          	at java.lang.Thread.run(Thread.java:745)
          
          
          Results :
          
          Tests in error: 
            STestS3AHugeFilesDiskBlocks>AbstractSTestS3AHugeFiles.test_010_CreateHugeFile:149 » AWSClientIO
          
          
          Show
          stevel@apache.org Steve Loughran added a comment - 2GB uploads when file buffered fails; AWS complains Running org.apache.hadoop.fs.s3a.scale.STestS3AHugeFilesDiskBlocks Tests run: 5, Failures: 0, Errors: 1, Skipped: 3, Time elapsed: 1,258.424 sec <<< FAILURE! - in org.apache.hadoop.fs.s3a.scale.STestS3AHugeFilesDiskBlocks test_010_CreateHugeFile(org.apache.hadoop.fs.s3a.scale.STestS3AHugeFilesDiskBlocks) Time elapsed: 1,256.013 sec <<< ERROR! org.apache.hadoop.fs.s3a.AWSClientIOException: Multi-part upload with id 'dZga.hig99Nxdm1S5dlcilzpg1kiav7ZF2QCJZZydN0qyE7U_pMUEYdACOavY_us3q9CgIxfKaQadXLhgUseUw--' on tests3a/scale/hugefile: com.amazonaws.ResetException: Failed to reset the request input stream; If the request involves an input stream, the maximum stream buffer size can be configured via request.getRequestClientOptions().setReadLimit( int ): Failed to reset the request input stream; If the request involves an input stream, the maximum stream buffer size can be configured via request.getRequestClientOptions().setReadLimit( int ) at org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:108) at org.apache.hadoop.fs.s3a.S3AUtils.extractException(S3AUtils.java:165) at org.apache.hadoop.fs.s3a.S3ABlockOutputStream$MultiPartUpload.waitForAllPartUploads(S3ABlockOutputStream.java:418) at org.apache.hadoop.fs.s3a.S3ABlockOutputStream$MultiPartUpload.access$100(S3ABlockOutputStream.java:356) at org.apache.hadoop.fs.s3a.S3ABlockOutputStream.close(S3ABlockOutputStream.java:261) at org.apache.hadoop.fs.FSDataOutputStream$PositionCache.close(FSDataOutputStream.java:72) at org.apache.hadoop.fs.FSDataOutputStream.close(FSDataOutputStream.java:106) at org.apache.hadoop.fs.s3a.scale.AbstractSTestS3AHugeFiles.test_010_CreateHugeFile(AbstractSTestS3AHugeFiles.java:149) 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: com.amazonaws.ResetException: Failed to reset the request input stream; If the request involves an input stream, the maximum stream buffer size can be configured via request.getRequestClientOptions().setReadLimit( int ) at com.amazonaws.http.AmazonHttpClient.executeOneRequest(AmazonHttpClient.java:665) at com.amazonaws.http.AmazonHttpClient.executeHelper(AmazonHttpClient.java:489) at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:310) at com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:3785) at com.amazonaws.services.s3.AmazonS3Client.doUploadPart(AmazonS3Client.java:2921) at com.amazonaws.services.s3.AmazonS3Client.uploadPart(AmazonS3Client.java:2906) at org.apache.hadoop.fs.s3a.S3AFileSystem.uploadPart(S3AFileSystem.java:1141) at org.apache.hadoop.fs.s3a.S3ABlockOutputStream$MultiPartUpload$1.call(S3ABlockOutputStream.java:391) at org.apache.hadoop.fs.s3a.S3ABlockOutputStream$MultiPartUpload$1.call(S3ABlockOutputStream.java:384) at org.apache.hadoop.fs.s3a.BlockingThreadPoolExecutorService$CallableWithPermitRelease.call(BlockingThreadPoolExecutorService.java:239) 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) at java.lang. Thread .run( Thread .java:745) Caused by: java.io.IOException: Resetting to invalid mark at java.io.BufferedInputStream.reset(BufferedInputStream.java:448) at org.apache.hadoop.fs.s3a.S3ADataBlocks$ForwardingInputStream.reset(S3ADataBlocks.java:432) at com.amazonaws.internal.SdkFilterInputStream.reset(SdkFilterInputStream.java:102) at com.amazonaws.internal.SdkFilterInputStream.reset(SdkFilterInputStream.java:102) at com.amazonaws.services.s3.internal.InputSubstream.reset(InputSubstream.java:110) at com.amazonaws.internal.SdkFilterInputStream.reset(SdkFilterInputStream.java:102) at com.amazonaws.services.s3.internal.MD5DigestCalculatingInputStream.reset(MD5DigestCalculatingInputStream.java:76) at com.amazonaws.internal.SdkFilterInputStream.reset(SdkFilterInputStream.java:102) at com.amazonaws.event.ProgressInputStream.reset(ProgressInputStream.java:139) at com.amazonaws.internal.SdkFilterInputStream.reset(SdkFilterInputStream.java:102) at com.amazonaws.http.AmazonHttpClient.executeOneRequest(AmazonHttpClient.java:663) at com.amazonaws.http.AmazonHttpClient.executeHelper(AmazonHttpClient.java:489) at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:310) at com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:3785) at com.amazonaws.services.s3.AmazonS3Client.doUploadPart(AmazonS3Client.java:2921) at com.amazonaws.services.s3.AmazonS3Client.uploadPart(AmazonS3Client.java:2906) at org.apache.hadoop.fs.s3a.S3AFileSystem.uploadPart(S3AFileSystem.java:1141) at org.apache.hadoop.fs.s3a.S3ABlockOutputStream$MultiPartUpload$1.call(S3ABlockOutputStream.java:391) at org.apache.hadoop.fs.s3a.S3ABlockOutputStream$MultiPartUpload$1.call(S3ABlockOutputStream.java:384) at org.apache.hadoop.fs.s3a.BlockingThreadPoolExecutorService$CallableWithPermitRelease.call(BlockingThreadPoolExecutorService.java:239) 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) at java.lang. Thread .run( Thread .java:745) Results : Tests in error: STestS3AHugeFilesDiskBlocks>AbstractSTestS3AHugeFiles.test_010_CreateHugeFile:149 » AWSClientIO
          Hide
          stevel@apache.org Steve Loughran added a comment -

          Looks related to this stack overflow topic and AWS issue 427

          Passing the file direct to the xfer manager apparently helps, though that will complicate the process in two ways:

          1. the buffering mechanism is now visible to the S3aBlockOutputStream
          2. there's the problem of deleting the file after the async xfer operation completes. Currently the stream deletes it in close(); without that a progress callback would need to react to the completed event and delete the file. Viable.

          before then: experiment without the buffering (performance impact?) and with smaller partition sizes.

          Also an unrelated idea: what about an option for always making the first block a memory block. That way, small files will be written without going near the local FS, but larger files will be uploaded.

          Show
          stevel@apache.org Steve Loughran added a comment - Looks related to this stack overflow topic and AWS issue 427 Passing the file direct to the xfer manager apparently helps, though that will complicate the process in two ways: 1. the buffering mechanism is now visible to the S3aBlockOutputStream 2. there's the problem of deleting the file after the async xfer operation completes. Currently the stream deletes it in close(); without that a progress callback would need to react to the completed event and delete the file. Viable. before then: experiment without the buffering (performance impact?) and with smaller partition sizes. Also an unrelated idea: what about an option for always making the first block a memory block. That way, small files will be written without going near the local FS, but larger files will be uploaded.
          Hide
          githubbot ASF GitHub Bot added a comment -

          GitHub user steveloughran opened a pull request:

          https://github.com/apache/hadoop/pull/125

          HADOOP-13560 S3A to support huge file writes and operations -with tests

          Adds

            1. Scale tests for S3A huge file support;
          • always running at the MB size (maybe best to make optional)
            -configurable to bigger sizes in the auth-keys XML or in the build `-Dfs.s3a.scale.test.huge.filesize=1000`
          • limited to upload, seek, read, rename, delete. The JUnit test cases are explicltly set up to run in order here.
            1. New scalable output stream for writing, `S3ABlockOutputStream`

          -always saves in incremental blocks as writes proceed, block size == partition size.
          -supports Fast output stream memory buffer code (for regression testing)
          -supports a back end which buffers blocks in files, using RR disk allocation. As such, write/read bandwidth is limited to aggregate HDD bandwidth.
          -adding extra failure resilience as testing throws up failure conditions (network timeouts, no-response from server on multipart commit, etc).
          -adding instrumentation, including using callbacks from AWS SDK to update gauges and counters (in progress)

          What we have here is essentially something that can replace the classic "save to file, upload at the end" stream and the fast "store it all in RAM and hope there's space" stream. It should offer incremental upload for faster output of larger files compared the classic file stream, with the scaleability the fast one lacks. And the instrumentation to show what's happening.

          You can merge this pull request into a Git repository by running:

          $ git pull https://github.com/steveloughran/hadoop s3/HADOOP-13560-5GB-blobs

          Alternatively you can review and apply these changes as the patch at:

          https://github.com/apache/hadoop/pull/125.patch

          To close this pull request, make a commit to your master/trunk branch
          with (at least) the following in the commit message:

          This closes #125


          commit d21600ead50aaafed1611b00206991c7d2c5934f
          Author: Steve Loughran <stevel@apache.org>
          Date: 2016-08-30T15:48:14Z

          HADOOP-13560 adding test for create/copy 5GB files

          commit 13b0544fffe7feb3e6d7404c90f222f1ae6644bb
          Author: Steve Loughran <stevel@apache.org>
          Date: 2016-08-31T11:29:28Z

          HADOOP-13560 tuning test scale and timeouts

          commit fb6a70c8d2b36c66d7b3ae732d9afd80b436a512
          Author: Steve Loughran <stevel@apache.org>
          Date: 2016-08-31T13:21:23Z

          HADOOP-13560 scale tests take maven build arguments

          commit d09aad6377fc37912d1c47355a191bc3279a4016
          Author: Steve Loughran <stevel@apache.org>
          Date: 2016-08-31T13:33:47Z

          HADOOP-13567 S3AFileSystem to override getStoragetStatistics() and so serve up its statistics

          commit e8afc25621e3552b80463084df29f785ecde6807
          Author: Steve Loughran <stevel@apache.org>
          Date: 2016-08-31T13:34:26Z

          HADOOP-13566 NPE in S3AFastOutputStream.write

          commit dfa90a08d18b7cda8c135ba8b838929a28784a47
          Author: Steve Loughran <stevel@apache.org>
          Date: 2016-08-31T14:31:39Z

          HADOOP-13560 use STest as prefix for scale tests

          commit 27365023e9363763c300e81bdefcb45887131ce4
          Author: Steve Loughran <stevel@apache.org>
          Date: 2016-08-31T17:06:51Z

          HADOOP-13560 test improvements

          commit a46781589ae8cedbdfeabb92fcc1ca83afc21b4c
          Author: Steve Loughran <stevel@apache.org>
          Date: 2016-08-31T17:07:29Z

          HADOOP-13560 fix typo in the name of a statistic

          commit cfdb0f0dbe8231a63046ba19900ea46645462bcb
          Author: Steve Loughran <stevel@apache.org>
          Date: 2016-08-31T17:08:42Z

          HADOOP-13569 S3AFastOutputStream to take ProgressListener in file create()

          commit 8ffd7a90fff7a5ed460b0396232d5322a06f8e59
          Author: Steve Loughran <stevel@apache.org>
          Date: 2016-09-01T17:05:01Z

          HADOOP-13560 lots of improvement in test and monitoring of what is going on inside S3A, including a gauge of active request counts. +more troubleshooting docs. The fast output stream will retry on errors

          commit 750e9462b7bd267915f9d91cbab0cd0ba51f1c41
          Author: Steve Loughran <stevel@apache.org>
          Date: 2016-08-28T11:38:40Z

          HADOOP-13531 S3 output stream allocator to round-robin directories

          commit 51c27278bcfa067109efa702deed6890db677895
          Author: Steve Loughran <stevel@apache.org>
          Date: 2016-09-05T17:48:36Z

          HADOOP-13560 WiP: adding new incremental output stream

          commit e1ce5a804a1c5d0afddf21362fe5a8d7d5179c58
          Author: Steve Loughran <stevel@apache.org>
          Date: 2016-09-06T13:46:50Z

          HADOOP-13560 data block design is coalescing and memory buffer writes are passing tests

          commit db1ed581b26c0320209017a09e77754638e7c42a
          Author: Steve Loughran <stevel@apache.org>
          Date: 2016-09-06T19:58:14Z

          HADOOP-13560 patch 002
          block streaming is in, testing at moderate scale <100 MB.

          you can choose for buffer-by-ram (current fast uploader) or buffer by HDD; in a test using SSD & remote S3, I got ~1.38MB/s bandwidth, got something similar 1.44 on RAM. But: we shouldn't run out off heap on the HDD option. RAM buffering uses existing ByteArrays, to ease source code migration off FastUpload (which is still there, for now).

          • I do plan to add pooled ByteBuffers
          • Add metrics of total and ongoing upload, including tracking what quantity of the outstanding block data has actually been uploaded.

          commit a068598c5c89e46f98ab05deb23e43d38556e424
          Author: Steve Loughran <stevel@apache.org>
          Date: 2016-09-07T14:12:11Z

          HADOOP-13560 ongoing work on disk uploads at 2+ GB scale.

          commit 9229c642a0380e6c8bb225e89d688fef1e9cb05c
          Author: Steve Loughran <stevel@apache.org>
          Date: 2016-09-07T15:12:16Z

          HADOOP-13560 complete merge with branch-2. Milestone: 1GB file round trip @ 1.57 MB/s


          Show
          githubbot ASF GitHub Bot added a comment - GitHub user steveloughran opened a pull request: https://github.com/apache/hadoop/pull/125 HADOOP-13560 S3A to support huge file writes and operations -with tests Adds Scale tests for S3A huge file support; always running at the MB size (maybe best to make optional) -configurable to bigger sizes in the auth-keys XML or in the build `-Dfs.s3a.scale.test.huge.filesize=1000` limited to upload, seek, read, rename, delete. The JUnit test cases are explicltly set up to run in order here. New scalable output stream for writing, `S3ABlockOutputStream` -always saves in incremental blocks as writes proceed, block size == partition size. -supports Fast output stream memory buffer code (for regression testing) -supports a back end which buffers blocks in files, using RR disk allocation. As such, write/read bandwidth is limited to aggregate HDD bandwidth. -adding extra failure resilience as testing throws up failure conditions (network timeouts, no-response from server on multipart commit, etc). -adding instrumentation, including using callbacks from AWS SDK to update gauges and counters (in progress) What we have here is essentially something that can replace the classic "save to file, upload at the end" stream and the fast "store it all in RAM and hope there's space" stream. It should offer incremental upload for faster output of larger files compared the classic file stream, with the scaleability the fast one lacks. And the instrumentation to show what's happening. You can merge this pull request into a Git repository by running: $ git pull https://github.com/steveloughran/hadoop s3/ HADOOP-13560 -5GB-blobs Alternatively you can review and apply these changes as the patch at: https://github.com/apache/hadoop/pull/125.patch To close this pull request, make a commit to your master/trunk branch with (at least) the following in the commit message: This closes #125 commit d21600ead50aaafed1611b00206991c7d2c5934f Author: Steve Loughran <stevel@apache.org> Date: 2016-08-30T15:48:14Z HADOOP-13560 adding test for create/copy 5GB files commit 13b0544fffe7feb3e6d7404c90f222f1ae6644bb Author: Steve Loughran <stevel@apache.org> Date: 2016-08-31T11:29:28Z HADOOP-13560 tuning test scale and timeouts commit fb6a70c8d2b36c66d7b3ae732d9afd80b436a512 Author: Steve Loughran <stevel@apache.org> Date: 2016-08-31T13:21:23Z HADOOP-13560 scale tests take maven build arguments commit d09aad6377fc37912d1c47355a191bc3279a4016 Author: Steve Loughran <stevel@apache.org> Date: 2016-08-31T13:33:47Z HADOOP-13567 S3AFileSystem to override getStoragetStatistics() and so serve up its statistics commit e8afc25621e3552b80463084df29f785ecde6807 Author: Steve Loughran <stevel@apache.org> Date: 2016-08-31T13:34:26Z HADOOP-13566 NPE in S3AFastOutputStream.write commit dfa90a08d18b7cda8c135ba8b838929a28784a47 Author: Steve Loughran <stevel@apache.org> Date: 2016-08-31T14:31:39Z HADOOP-13560 use STest as prefix for scale tests commit 27365023e9363763c300e81bdefcb45887131ce4 Author: Steve Loughran <stevel@apache.org> Date: 2016-08-31T17:06:51Z HADOOP-13560 test improvements commit a46781589ae8cedbdfeabb92fcc1ca83afc21b4c Author: Steve Loughran <stevel@apache.org> Date: 2016-08-31T17:07:29Z HADOOP-13560 fix typo in the name of a statistic commit cfdb0f0dbe8231a63046ba19900ea46645462bcb Author: Steve Loughran <stevel@apache.org> Date: 2016-08-31T17:08:42Z HADOOP-13569 S3AFastOutputStream to take ProgressListener in file create() commit 8ffd7a90fff7a5ed460b0396232d5322a06f8e59 Author: Steve Loughran <stevel@apache.org> Date: 2016-09-01T17:05:01Z HADOOP-13560 lots of improvement in test and monitoring of what is going on inside S3A, including a gauge of active request counts. +more troubleshooting docs. The fast output stream will retry on errors commit 750e9462b7bd267915f9d91cbab0cd0ba51f1c41 Author: Steve Loughran <stevel@apache.org> Date: 2016-08-28T11:38:40Z HADOOP-13531 S3 output stream allocator to round-robin directories commit 51c27278bcfa067109efa702deed6890db677895 Author: Steve Loughran <stevel@apache.org> Date: 2016-09-05T17:48:36Z HADOOP-13560 WiP: adding new incremental output stream commit e1ce5a804a1c5d0afddf21362fe5a8d7d5179c58 Author: Steve Loughran <stevel@apache.org> Date: 2016-09-06T13:46:50Z HADOOP-13560 data block design is coalescing and memory buffer writes are passing tests commit db1ed581b26c0320209017a09e77754638e7c42a Author: Steve Loughran <stevel@apache.org> Date: 2016-09-06T19:58:14Z HADOOP-13560 patch 002 block streaming is in, testing at moderate scale <100 MB. you can choose for buffer-by-ram (current fast uploader) or buffer by HDD; in a test using SSD & remote S3, I got ~1.38MB/s bandwidth, got something similar 1.44 on RAM. But: we shouldn't run out off heap on the HDD option. RAM buffering uses existing ByteArrays, to ease source code migration off FastUpload (which is still there, for now). I do plan to add pooled ByteBuffers Add metrics of total and ongoing upload, including tracking what quantity of the outstanding block data has actually been uploaded. commit a068598c5c89e46f98ab05deb23e43d38556e424 Author: Steve Loughran <stevel@apache.org> Date: 2016-09-07T14:12:11Z HADOOP-13560 ongoing work on disk uploads at 2+ GB scale. commit 9229c642a0380e6c8bb225e89d688fef1e9cb05c Author: Steve Loughran <stevel@apache.org> Date: 2016-09-07T15:12:16Z HADOOP-13560 complete merge with branch-2. Milestone: 1GB file round trip @ 1.57 MB/s
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user steveloughran commented on a diff in the pull request:

          https://github.com/apache/hadoop/pull/125#discussion_r77982045

          — Diff: hadoop-tools/hadoop-aws/pom.xml —
          @@ -183,6 +199,8 @@
          <include>**/ITestS3AFileSystemContract.java</include>
          <include>**/ITestS3AMiniYarnCluster.java</include>
          <include>*/ITest*Root.java</include>
          + <include>**/ITestS3AFileContextStatistics.java</include>
          — End diff –

          moved this line down as it was failing sometimes

          Show
          githubbot ASF GitHub Bot added a comment - Github user steveloughran commented on a diff in the pull request: https://github.com/apache/hadoop/pull/125#discussion_r77982045 — Diff: hadoop-tools/hadoop-aws/pom.xml — @@ -183,6 +199,8 @@ <include>**/ITestS3AFileSystemContract.java</include> <include>**/ITestS3AMiniYarnCluster.java</include> <include>* /ITest*Root .java</include> + <include>**/ITestS3AFileContextStatistics.java</include> — End diff – moved this line down as it was failing sometimes
          Hide
          stevel@apache.org Steve Loughran added a comment -

          New code adds stats too; here is the tail end of an 80 MB upload

          Statistics: OutputStreamStatistics{blocksSubmitted=400, blocksInQueue=0, blocksActive=0, blockUploadsCompleted=400, blockUploadsFailed=0, bytesPendingUpload=0, bytesUploaded=2097152000, transferDuration=13308123 ms, queueDuration=26935221 ms, averageQueueTime=67338 ms, totalUploadDuration=40243344 ms, effectiveBandwidth=52111.77281887907 bytes/s}
          

          that bandwith measure is low as includes queue time, and there are many blocks from the same stream then the aggregate queue time is pretty high. The real B/W here is :

          2016-09-08 13:35:52,185 [JUnit] INFO  scale.AbstractSTestS3AHugeFiles (AbstractSTestS3AHugeFiles.java:test_010_CreateHugeFile(157)) - Time per MB to write = 669,477,018 nS
          2016-09-08 13:35:52,186 [JUnit] INFO  scale.AbstractSTestS3AHugeFiles (AbstractSTestS3AHugeFiles.java:test_010_CreateHugeFile(159)) - Effective Bandwidth: 1.5662613821040523 MB/s
          2016-09-08 13:35:52,186 [JUnit] INFO  scale.AbstractSTestS3AHugeFiles (AbstractSTestS3AHugeFiles.java:test_010_CreateHugeFile(162)) - PUT 2097152000 bytes in 400 operations; 5 MB/operation
          2016-09-08 13:35:52,186 [JUnit] INFO  scale.AbstractSTestS3AHugeFiles (AbstractSTestS3AHugeFiles.java:test_010_CreateHugeFile(165)) - Time per PUT 3,347,385,091 nS
          

          That queue duration also includes time that the thread generating the output is blocked awaiting submission of work. As that submission is happening in a sync block, I worry that this blocking will make this output stream (and the FastOutputStream) something that can't be interrupted easily. Does that matter?

          Show
          stevel@apache.org Steve Loughran added a comment - New code adds stats too; here is the tail end of an 80 MB upload Statistics: OutputStreamStatistics{blocksSubmitted=400, blocksInQueue=0, blocksActive=0, blockUploadsCompleted=400, blockUploadsFailed=0, bytesPendingUpload=0, bytesUploaded=2097152000, transferDuration=13308123 ms, queueDuration=26935221 ms, averageQueueTime=67338 ms, totalUploadDuration=40243344 ms, effectiveBandwidth=52111.77281887907 bytes/s} that bandwith measure is low as includes queue time, and there are many blocks from the same stream then the aggregate queue time is pretty high. The real B/W here is : 2016-09-08 13:35:52,185 [JUnit] INFO scale.AbstractSTestS3AHugeFiles (AbstractSTestS3AHugeFiles.java:test_010_CreateHugeFile(157)) - Time per MB to write = 669,477,018 nS 2016-09-08 13:35:52,186 [JUnit] INFO scale.AbstractSTestS3AHugeFiles (AbstractSTestS3AHugeFiles.java:test_010_CreateHugeFile(159)) - Effective Bandwidth: 1.5662613821040523 MB/s 2016-09-08 13:35:52,186 [JUnit] INFO scale.AbstractSTestS3AHugeFiles (AbstractSTestS3AHugeFiles.java:test_010_CreateHugeFile(162)) - PUT 2097152000 bytes in 400 operations; 5 MB/operation 2016-09-08 13:35:52,186 [JUnit] INFO scale.AbstractSTestS3AHugeFiles (AbstractSTestS3AHugeFiles.java:test_010_CreateHugeFile(165)) - Time per PUT 3,347,385,091 nS That queue duration also includes time that the thread generating the output is blocked awaiting submission of work. As that submission is happening in a sync block, I worry that this blocking will make this output stream (and the FastOutputStream) something that can't be interrupted easily. Does that matter?
          Hide
          stevel@apache.org Steve Loughran added a comment -

          latest PR update includes documentation.

          One feature we could also consider is caching write failures in the background threads and raising them in the output stream thread once one one has been caught and logged. This will not let the caller re-attempt the write but it will make it visible, and it will make it visible fast.

          As it is, failures are delayed until the close operation and waitForAllPartUploads(), which is at risk of having the exception swallowed, or, at best, logged —so risking hiding the fact that a write has failed and that data has been lost.

          Show
          stevel@apache.org Steve Loughran added a comment - latest PR update includes documentation. One feature we could also consider is caching write failures in the background threads and raising them in the output stream thread once one one has been caught and logged. This will not let the caller re-attempt the write but it will make it visible, and it will make it visible fast . As it is, failures are delayed until the close operation and waitForAllPartUploads(), which is at risk of having the exception swallowed, or, at best, logged —so risking hiding the fact that a write has failed and that data has been lost.
          Hide
          stevel@apache.org Steve Loughran added a comment -

          Patch 003

          • (pooled) ByteBuffer now an option for buffering output, this should offer a in-memory performance with less risk of heap overflow. But it can still use enough memory that your Yarn hosted JVMs get killed; it's still only to be used with care
          • replaced S3AFastOutputStream. The option is deprecated and downgraded to buffered + file.
          • Pulled all fast output streams tests but a little one to verify that the options still work.
          • I've not deleted the S3AFastOutputStream class —yet. It's there for comparing new vs. old
          • javadocs in more places
          • core-default.xml descriptions improved
          • index.md updated with new values, more text
          • tests pass down scale test maven options to sequential test runs.

          Test endpoint: S3 ireland

          I think this code is ready for review/testing by others. Can anyone doing this start with the documentation to see if it explains it, then go into the code. Ideally I'd like some testing of large distcps with the file buffering (verifies it scales) and the bytebuffer (to see how it fails, and add it to the troubleshooting docs)

          Show
          stevel@apache.org Steve Loughran added a comment - Patch 003 (pooled) ByteBuffer now an option for buffering output, this should offer a in-memory performance with less risk of heap overflow. But it can still use enough memory that your Yarn hosted JVMs get killed; it's still only to be used with care replaced S3AFastOutputStream. The option is deprecated and downgraded to buffered + file. Pulled all fast output streams tests but a little one to verify that the options still work. I've not deleted the S3AFastOutputStream class —yet. It's there for comparing new vs. old javadocs in more places core-default.xml descriptions improved index.md updated with new values, more text tests pass down scale test maven options to sequential test runs. Test endpoint: S3 ireland I think this code is ready for review/testing by others. Can anyone doing this start with the documentation to see if it explains it, then go into the code. Ideally I'd like some testing of large distcps with the file buffering (verifies it scales) and the bytebuffer (to see how it fails, and add it to the troubleshooting docs)
          Hide
          stevel@apache.org Steve Loughran added a comment - - edited

          Patch 004

          • fixed name of fs.s3a.block.output option in core-default and docs. Thanks Rajesh!
          • more attempts at managing close() operation rigorously. No evidence this is the cause of the problem rajesh saw though.
          • rearranged layout of code in S3ADatablocks so associated classes are adjacent;
          • retry on multipart commit adding sleep statements between retries
          • gauges of active block uploads wired up.
          • more debug statements
          • new Progress log for logging progress @ debug level in s3a. Why? Because logging events every 8KB gets too chatty when debugging many-MB uploads.

          test: s3a ireland

          Show
          stevel@apache.org Steve Loughran added a comment - - edited Patch 004 fixed name of fs.s3a.block.output option in core-default and docs. Thanks Rajesh! more attempts at managing close() operation rigorously. No evidence this is the cause of the problem rajesh saw though. rearranged layout of code in S3ADatablocks so associated classes are adjacent; retry on multipart commit adding sleep statements between retries gauges of active block uploads wired up. more debug statements new Progress log for logging progress @ debug level in s3a. Why? Because logging events every 8KB gets too chatty when debugging many-MB uploads. test: s3a ireland
          Hide
          hadoopqa Hadoop QA added a comment -
          -1 overall



          Vote Subsystem Runtime Comment
          0 reexec 16m 6s Docker mode activated.
          +1 @author 0m 0s The patch does not contain any @author tags.
          +1 test4tests 0m 0s The patch appears to include 23 new or modified test files.
          0 mvndep 1m 12s Maven dependency ordering for branch
          +1 mvninstall 7m 22s branch-2 passed
          +1 compile 7m 7s branch-2 passed with JDK v1.8.0_101
          +1 compile 7m 41s branch-2 passed with JDK v1.7.0_111
          +1 checkstyle 1m 36s branch-2 passed
          +1 mvnsite 1m 30s branch-2 passed
          +1 mvneclipse 0m 43s branch-2 passed
          +1 findbugs 2m 29s branch-2 passed
          +1 javadoc 1m 10s branch-2 passed with JDK v1.8.0_101
          +1 javadoc 1m 18s branch-2 passed with JDK v1.7.0_111
          0 mvndep 0m 17s Maven dependency ordering for patch
          -1 mvninstall 0m 13s hadoop-aws in the patch failed.
          -1 compile 6m 48s root in the patch failed with JDK v1.8.0_101.
          -1 javac 6m 48s root in the patch failed with JDK v1.8.0_101.
          -1 compile 7m 9s root in the patch failed with JDK v1.7.0_111.
          -1 javac 7m 9s root in the patch failed with JDK v1.7.0_111.
          -0 checkstyle 1m 33s root: The patch generated 73 new + 32 unchanged - 1 fixed = 105 total (was 33)
          -1 mvnsite 0m 18s hadoop-aws in the patch failed.
          +1 mvneclipse 0m 36s the patch passed
          -1 whitespace 0m 0s The patch has 63 line(s) that end in whitespace. Use git apply --whitespace=fix <<patch_file>>. Refer https://git-scm.com/docs/git-apply
          -1 whitespace 0m 1s The patch 36 line(s) with tabs.
          +1 xml 0m 1s The patch has no ill-formed XML file.
          -1 findbugs 0m 20s hadoop-aws in the patch failed.
          -1 javadoc 0m 28s hadoop-aws in the patch failed with JDK v1.8.0_101.
          -1 javadoc 0m 18s hadoop-aws in the patch failed with JDK v1.7.0_111.
          +1 unit 9m 54s hadoop-common in the patch passed with JDK v1.7.0_111.
          -1 unit 0m 15s hadoop-aws in the patch failed with JDK v1.7.0_111.
          +1 asflicense 0m 24s The patch does not generate ASF License warnings.
          116m 47s



          Subsystem Report/Notes
          Docker Image:yetus/hadoop:b59b8b7
          JIRA Issue HADOOP-13560
          GITHUB PR https://github.com/apache/hadoop/pull/125
          Optional Tests asflicense compile javac javadoc mvninstall mvnsite unit xml findbugs checkstyle
          uname Linux 4cf4f66e23f7 3.13.0-93-generic #140-Ubuntu SMP Mon Jul 18 21:21:05 UTC 2016 x86_64 x86_64 x86_64 GNU/Linux
          Build tool maven
          Personality /testptch/hadoop/patchprocess/precommit/personality/provided.sh
          git revision branch-2 / 1b91ebb
          Default Java 1.7.0_111
          Multi-JDK versions /usr/lib/jvm/java-8-oracle:1.8.0_101 /usr/lib/jvm/java-7-openjdk-amd64:1.7.0_111
          findbugs v3.0.0
          mvninstall https://builds.apache.org/job/PreCommit-HADOOP-Build/10561/artifact/patchprocess/patch-mvninstall-hadoop-tools_hadoop-aws.txt
          compile https://builds.apache.org/job/PreCommit-HADOOP-Build/10561/artifact/patchprocess/patch-compile-root-jdk1.8.0_101.txt
          javac https://builds.apache.org/job/PreCommit-HADOOP-Build/10561/artifact/patchprocess/patch-compile-root-jdk1.8.0_101.txt
          compile https://builds.apache.org/job/PreCommit-HADOOP-Build/10561/artifact/patchprocess/patch-compile-root-jdk1.7.0_111.txt
          javac https://builds.apache.org/job/PreCommit-HADOOP-Build/10561/artifact/patchprocess/patch-compile-root-jdk1.7.0_111.txt
          checkstyle https://builds.apache.org/job/PreCommit-HADOOP-Build/10561/artifact/patchprocess/diff-checkstyle-root.txt
          mvnsite https://builds.apache.org/job/PreCommit-HADOOP-Build/10561/artifact/patchprocess/patch-mvnsite-hadoop-tools_hadoop-aws.txt
          whitespace https://builds.apache.org/job/PreCommit-HADOOP-Build/10561/artifact/patchprocess/whitespace-eol.txt
          whitespace https://builds.apache.org/job/PreCommit-HADOOP-Build/10561/artifact/patchprocess/whitespace-tabs.txt
          findbugs https://builds.apache.org/job/PreCommit-HADOOP-Build/10561/artifact/patchprocess/patch-findbugs-hadoop-tools_hadoop-aws.txt
          javadoc https://builds.apache.org/job/PreCommit-HADOOP-Build/10561/artifact/patchprocess/patch-javadoc-hadoop-tools_hadoop-aws-jdk1.8.0_101.txt
          javadoc https://builds.apache.org/job/PreCommit-HADOOP-Build/10561/artifact/patchprocess/patch-javadoc-hadoop-tools_hadoop-aws-jdk1.7.0_111.txt
          unit https://builds.apache.org/job/PreCommit-HADOOP-Build/10561/artifact/patchprocess/patch-unit-hadoop-tools_hadoop-aws-jdk1.7.0_111.txt
          JDK v1.7.0_111 Test Results https://builds.apache.org/job/PreCommit-HADOOP-Build/10561/testReport/
          modules C: hadoop-common-project/hadoop-common hadoop-tools/hadoop-aws U: .
          Console output https://builds.apache.org/job/PreCommit-HADOOP-Build/10561/console
          Powered by Apache Yetus 0.4.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 16m 6s Docker mode activated. +1 @author 0m 0s The patch does not contain any @author tags. +1 test4tests 0m 0s The patch appears to include 23 new or modified test files. 0 mvndep 1m 12s Maven dependency ordering for branch +1 mvninstall 7m 22s branch-2 passed +1 compile 7m 7s branch-2 passed with JDK v1.8.0_101 +1 compile 7m 41s branch-2 passed with JDK v1.7.0_111 +1 checkstyle 1m 36s branch-2 passed +1 mvnsite 1m 30s branch-2 passed +1 mvneclipse 0m 43s branch-2 passed +1 findbugs 2m 29s branch-2 passed +1 javadoc 1m 10s branch-2 passed with JDK v1.8.0_101 +1 javadoc 1m 18s branch-2 passed with JDK v1.7.0_111 0 mvndep 0m 17s Maven dependency ordering for patch -1 mvninstall 0m 13s hadoop-aws in the patch failed. -1 compile 6m 48s root in the patch failed with JDK v1.8.0_101. -1 javac 6m 48s root in the patch failed with JDK v1.8.0_101. -1 compile 7m 9s root in the patch failed with JDK v1.7.0_111. -1 javac 7m 9s root in the patch failed with JDK v1.7.0_111. -0 checkstyle 1m 33s root: The patch generated 73 new + 32 unchanged - 1 fixed = 105 total (was 33) -1 mvnsite 0m 18s hadoop-aws in the patch failed. +1 mvneclipse 0m 36s the patch passed -1 whitespace 0m 0s The patch has 63 line(s) that end in whitespace. Use git apply --whitespace=fix <<patch_file>>. Refer https://git-scm.com/docs/git-apply -1 whitespace 0m 1s The patch 36 line(s) with tabs. +1 xml 0m 1s The patch has no ill-formed XML file. -1 findbugs 0m 20s hadoop-aws in the patch failed. -1 javadoc 0m 28s hadoop-aws in the patch failed with JDK v1.8.0_101. -1 javadoc 0m 18s hadoop-aws in the patch failed with JDK v1.7.0_111. +1 unit 9m 54s hadoop-common in the patch passed with JDK v1.7.0_111. -1 unit 0m 15s hadoop-aws in the patch failed with JDK v1.7.0_111. +1 asflicense 0m 24s The patch does not generate ASF License warnings. 116m 47s Subsystem Report/Notes Docker Image:yetus/hadoop:b59b8b7 JIRA Issue HADOOP-13560 GITHUB PR https://github.com/apache/hadoop/pull/125 Optional Tests asflicense compile javac javadoc mvninstall mvnsite unit xml findbugs checkstyle uname Linux 4cf4f66e23f7 3.13.0-93-generic #140-Ubuntu SMP Mon Jul 18 21:21:05 UTC 2016 x86_64 x86_64 x86_64 GNU/Linux Build tool maven Personality /testptch/hadoop/patchprocess/precommit/personality/provided.sh git revision branch-2 / 1b91ebb Default Java 1.7.0_111 Multi-JDK versions /usr/lib/jvm/java-8-oracle:1.8.0_101 /usr/lib/jvm/java-7-openjdk-amd64:1.7.0_111 findbugs v3.0.0 mvninstall https://builds.apache.org/job/PreCommit-HADOOP-Build/10561/artifact/patchprocess/patch-mvninstall-hadoop-tools_hadoop-aws.txt compile https://builds.apache.org/job/PreCommit-HADOOP-Build/10561/artifact/patchprocess/patch-compile-root-jdk1.8.0_101.txt javac https://builds.apache.org/job/PreCommit-HADOOP-Build/10561/artifact/patchprocess/patch-compile-root-jdk1.8.0_101.txt compile https://builds.apache.org/job/PreCommit-HADOOP-Build/10561/artifact/patchprocess/patch-compile-root-jdk1.7.0_111.txt javac https://builds.apache.org/job/PreCommit-HADOOP-Build/10561/artifact/patchprocess/patch-compile-root-jdk1.7.0_111.txt checkstyle https://builds.apache.org/job/PreCommit-HADOOP-Build/10561/artifact/patchprocess/diff-checkstyle-root.txt mvnsite https://builds.apache.org/job/PreCommit-HADOOP-Build/10561/artifact/patchprocess/patch-mvnsite-hadoop-tools_hadoop-aws.txt whitespace https://builds.apache.org/job/PreCommit-HADOOP-Build/10561/artifact/patchprocess/whitespace-eol.txt whitespace https://builds.apache.org/job/PreCommit-HADOOP-Build/10561/artifact/patchprocess/whitespace-tabs.txt findbugs https://builds.apache.org/job/PreCommit-HADOOP-Build/10561/artifact/patchprocess/patch-findbugs-hadoop-tools_hadoop-aws.txt javadoc https://builds.apache.org/job/PreCommit-HADOOP-Build/10561/artifact/patchprocess/patch-javadoc-hadoop-tools_hadoop-aws-jdk1.8.0_101.txt javadoc https://builds.apache.org/job/PreCommit-HADOOP-Build/10561/artifact/patchprocess/patch-javadoc-hadoop-tools_hadoop-aws-jdk1.7.0_111.txt unit https://builds.apache.org/job/PreCommit-HADOOP-Build/10561/artifact/patchprocess/patch-unit-hadoop-tools_hadoop-aws-jdk1.7.0_111.txt JDK v1.7.0_111 Test Results https://builds.apache.org/job/PreCommit-HADOOP-Build/10561/testReport/ modules C: hadoop-common-project/hadoop-common hadoop-tools/hadoop-aws U: . Console output https://builds.apache.org/job/PreCommit-HADOOP-Build/10561/console Powered by Apache Yetus 0.4.0-SNAPSHOT http://yetus.apache.org This message was automatically generated.
          Hide
          stevel@apache.org Steve Loughran added a comment -

          Javac failing

          /testptch/hadoop/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AIncrementalOutputStream.java:[245,6] error: 'else' without 'if'
          

          Which is interesting because: The current patch and tip of PR doesn't have such a class.

          maybe Allen Wittenauer can tell me how I've got yetus confused. In the meantime I'll do a superficial change and update the PR to see what happens

          Show
          stevel@apache.org Steve Loughran added a comment - Javac failing /testptch/hadoop/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AIncrementalOutputStream.java:[245,6] error: ' else ' without ' if ' Which is interesting because: The current patch and tip of PR doesn't have such a class. maybe Allen Wittenauer can tell me how I've got yetus confused. In the meantime I'll do a superficial change and update the PR to see what happens
          Hide
          aw Allen Wittenauer added a comment -

          Squash your commits. If you look at the .patch file generated (https://patch-diff.githubusercontent.com/raw/apache/hadoop/pull/125.patch) S3AIncrementalOutputStream.java is definitely there.

          Before the question gets asked, Yetus can't use the .diff version (with all the merge bits resolved) because the .diff version doesn't include binary artifacts.

          Show
          aw Allen Wittenauer added a comment - Squash your commits. If you look at the .patch file generated ( https://patch-diff.githubusercontent.com/raw/apache/hadoop/pull/125.patch ) S3AIncrementalOutputStream.java is definitely there. Before the question gets asked, Yetus can't use the .diff version (with all the merge bits resolved) because the .diff version doesn't include binary artifacts.
          Hide
          githubbot ASF GitHub Bot added a comment -

          GitHub user steveloughran opened a pull request:

          https://github.com/apache/hadoop/pull/130

          HADOOP-13560 block output streams

          Merge commit of latest code.

          Docs, XML configs up to speed
          scale tests only run with a -Pscale option.
          some props can be configured in POM

          You can merge this pull request into a Git repository by running:

          $ git pull https://github.com/steveloughran/hadoop s3/HADOOP-13560-huge-blocks

          Alternatively you can review and apply these changes as the patch at:

          https://github.com/apache/hadoop/pull/130.patch

          To close this pull request, make a commit to your master/trunk branch
          with (at least) the following in the commit message:

          This closes #130


          commit 6183169f35629fb04490447b7b72d2a54097b3c5
          Author: Steve Loughran <stevel@apache.org>
          Date: 2016-09-23T18:57:25Z

          HADOOP-13560: squashl merge of the block output code


          Show
          githubbot ASF GitHub Bot added a comment - GitHub user steveloughran opened a pull request: https://github.com/apache/hadoop/pull/130 HADOOP-13560 block output streams Merge commit of latest code. Docs, XML configs up to speed scale tests only run with a -Pscale option. some props can be configured in POM You can merge this pull request into a Git repository by running: $ git pull https://github.com/steveloughran/hadoop s3/ HADOOP-13560 -huge-blocks Alternatively you can review and apply these changes as the patch at: https://github.com/apache/hadoop/pull/130.patch To close this pull request, make a commit to your master/trunk branch with (at least) the following in the commit message: This closes #130 commit 6183169f35629fb04490447b7b72d2a54097b3c5 Author: Steve Loughran <stevel@apache.org> Date: 2016-09-23T18:57:25Z HADOOP-13560 : squashl merge of the block output code
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user steveloughran closed the pull request at:

          https://github.com/apache/hadoop/pull/125

          Show
          githubbot ASF GitHub Bot added a comment - Github user steveloughran closed the pull request at: https://github.com/apache/hadoop/pull/125
          Hide
          stevel@apache.org Steve Loughran added a comment -

          1. fixed with new PR
          2. I didn't know enough about git to begin asking question #2

          Show
          stevel@apache.org Steve Loughran added a comment - 1. fixed with new PR 2. I didn't know enough about git to begin asking question #2
          Hide
          hadoopqa Hadoop QA added a comment -
          -1 overall



          Vote Subsystem Runtime Comment
          0 reexec 0m 21s Docker mode activated.
          +1 @author 0m 0s The patch does not contain any @author tags.
          +1 test4tests 0m 0s The patch appears to include 22 new or modified test files.
          0 mvndep 1m 6s Maven dependency ordering for branch
          +1 mvninstall 6m 41s branch-2 passed
          +1 compile 5m 33s branch-2 passed with JDK v1.8.0_101
          +1 compile 6m 25s branch-2 passed with JDK v1.7.0_111
          +1 checkstyle 1m 27s branch-2 passed
          +1 mvnsite 1m 21s branch-2 passed
          +1 mvneclipse 0m 30s branch-2 passed
          +1 findbugs 2m 12s branch-2 passed
          +1 javadoc 1m 1s branch-2 passed with JDK v1.8.0_101
          +1 javadoc 1m 15s branch-2 passed with JDK v1.7.0_111
          0 mvndep 0m 16s Maven dependency ordering for patch
          +1 mvninstall 1m 2s the patch passed
          +1 compile 5m 26s the patch passed with JDK v1.8.0_101
          -1 javac 5m 26s root-jdk1.8.0_101 with JDK v1.8.0_101 generated 4 new + 857 unchanged - 0 fixed = 861 total (was 857)
          +1 compile 6m 30s the patch passed with JDK v1.7.0_111
          -1 javac 6m 30s root-jdk1.7.0_111 with JDK v1.7.0_111 generated 5 new + 949 unchanged - 1 fixed = 954 total (was 950)
          -0 checkstyle 1m 29s root: The patch generated 56 new + 31 unchanged - 2 fixed = 87 total (was 33)
          +1 mvnsite 1m 28s the patch passed
          +1 mvneclipse 0m 37s the patch passed
          -1 whitespace 0m 0s The patch has 65 line(s) that end in whitespace. Use git apply --whitespace=fix <<patch_file>>. Refer https://git-scm.com/docs/git-apply
          -1 whitespace 0m 2s The patch 36 line(s) with tabs.
          +1 xml 0m 1s The patch has no ill-formed XML file.
          -1 findbugs 0m 52s hadoop-tools/hadoop-aws generated 2 new + 0 unchanged - 0 fixed = 2 total (was 0)
          +1 javadoc 1m 7s the patch passed with JDK v1.8.0_101
          +1 javadoc 1m 22s the patch passed with JDK v1.7.0_111
          +1 unit 8m 20s hadoop-common in the patch passed with JDK v1.7.0_111.
          +1 unit 0m 28s hadoop-aws in the patch passed with JDK v1.7.0_111.
          +1 asflicense 0m 29s The patch does not generate ASF License warnings.
          93m 12s



          Reason Tests
          FindBugs module:hadoop-tools/hadoop-aws
            Uninitialized read of activeBlock in new org.apache.hadoop.fs.s3a.S3ABlockOutputStream(S3AFileSystem, String, Progressable, long, S3ADataBlocks$BlockFactory, S3AInstrumentation$OutputStreamStatistics) At S3ABlockOutputStream.java:new org.apache.hadoop.fs.s3a.S3ABlockOutputStream(S3AFileSystem, String, Progressable, long, S3ADataBlocks$BlockFactory, S3AInstrumentation$OutputStreamStatistics) At S3ABlockOutputStream.java:[line 163]
            Exceptional return value of java.io.File.delete() ignored in org.apache.hadoop.fs.s3a.S3ADataBlocks$DiskBlock$FileDeletingInputStream.close() At S3ADataBlocks.java:ignored in org.apache.hadoop.fs.s3a.S3ADataBlocks$DiskBlock$FileDeletingInputStream.close() At S3ADataBlocks.java:[line 787]



          Subsystem Report/Notes
          Docker Image:yetus/hadoop:b59b8b7
          JIRA Issue HADOOP-13560
          GITHUB PR https://github.com/apache/hadoop/pull/130
          Optional Tests asflicense compile javac javadoc mvninstall mvnsite unit xml findbugs checkstyle
          uname Linux 1e87f6c4e839 3.13.0-93-generic #140-Ubuntu SMP Mon Jul 18 21:21:05 UTC 2016 x86_64 x86_64 x86_64 GNU/Linux
          Build tool maven
          Personality /testptch/hadoop/patchprocess/precommit/personality/provided.sh
          git revision branch-2 / 6c98455
          Default Java 1.7.0_111
          Multi-JDK versions /usr/lib/jvm/java-8-oracle:1.8.0_101 /usr/lib/jvm/java-7-openjdk-amd64:1.7.0_111
          findbugs v3.0.0
          javac https://builds.apache.org/job/PreCommit-HADOOP-Build/10583/artifact/patchprocess/diff-compile-javac-root-jdk1.8.0_101.txt
          javac https://builds.apache.org/job/PreCommit-HADOOP-Build/10583/artifact/patchprocess/diff-compile-javac-root-jdk1.7.0_111.txt
          checkstyle https://builds.apache.org/job/PreCommit-HADOOP-Build/10583/artifact/patchprocess/diff-checkstyle-root.txt
          whitespace https://builds.apache.org/job/PreCommit-HADOOP-Build/10583/artifact/patchprocess/whitespace-eol.txt
          whitespace https://builds.apache.org/job/PreCommit-HADOOP-Build/10583/artifact/patchprocess/whitespace-tabs.txt
          findbugs https://builds.apache.org/job/PreCommit-HADOOP-Build/10583/artifact/patchprocess/new-findbugs-hadoop-tools_hadoop-aws.html
          JDK v1.7.0_111 Test Results https://builds.apache.org/job/PreCommit-HADOOP-Build/10583/testReport/
          modules C: hadoop-common-project/hadoop-common hadoop-tools/hadoop-aws U: .
          Console output https://builds.apache.org/job/PreCommit-HADOOP-Build/10583/console
          Powered by Apache Yetus 0.4.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 21s Docker mode activated. +1 @author 0m 0s The patch does not contain any @author tags. +1 test4tests 0m 0s The patch appears to include 22 new or modified test files. 0 mvndep 1m 6s Maven dependency ordering for branch +1 mvninstall 6m 41s branch-2 passed +1 compile 5m 33s branch-2 passed with JDK v1.8.0_101 +1 compile 6m 25s branch-2 passed with JDK v1.7.0_111 +1 checkstyle 1m 27s branch-2 passed +1 mvnsite 1m 21s branch-2 passed +1 mvneclipse 0m 30s branch-2 passed +1 findbugs 2m 12s branch-2 passed +1 javadoc 1m 1s branch-2 passed with JDK v1.8.0_101 +1 javadoc 1m 15s branch-2 passed with JDK v1.7.0_111 0 mvndep 0m 16s Maven dependency ordering for patch +1 mvninstall 1m 2s the patch passed +1 compile 5m 26s the patch passed with JDK v1.8.0_101 -1 javac 5m 26s root-jdk1.8.0_101 with JDK v1.8.0_101 generated 4 new + 857 unchanged - 0 fixed = 861 total (was 857) +1 compile 6m 30s the patch passed with JDK v1.7.0_111 -1 javac 6m 30s root-jdk1.7.0_111 with JDK v1.7.0_111 generated 5 new + 949 unchanged - 1 fixed = 954 total (was 950) -0 checkstyle 1m 29s root: The patch generated 56 new + 31 unchanged - 2 fixed = 87 total (was 33) +1 mvnsite 1m 28s the patch passed +1 mvneclipse 0m 37s the patch passed -1 whitespace 0m 0s The patch has 65 line(s) that end in whitespace. Use git apply --whitespace=fix <<patch_file>>. Refer https://git-scm.com/docs/git-apply -1 whitespace 0m 2s The patch 36 line(s) with tabs. +1 xml 0m 1s The patch has no ill-formed XML file. -1 findbugs 0m 52s hadoop-tools/hadoop-aws generated 2 new + 0 unchanged - 0 fixed = 2 total (was 0) +1 javadoc 1m 7s the patch passed with JDK v1.8.0_101 +1 javadoc 1m 22s the patch passed with JDK v1.7.0_111 +1 unit 8m 20s hadoop-common in the patch passed with JDK v1.7.0_111. +1 unit 0m 28s hadoop-aws in the patch passed with JDK v1.7.0_111. +1 asflicense 0m 29s The patch does not generate ASF License warnings. 93m 12s Reason Tests FindBugs module:hadoop-tools/hadoop-aws   Uninitialized read of activeBlock in new org.apache.hadoop.fs.s3a.S3ABlockOutputStream(S3AFileSystem, String, Progressable, long, S3ADataBlocks$BlockFactory, S3AInstrumentation$OutputStreamStatistics) At S3ABlockOutputStream.java:new org.apache.hadoop.fs.s3a.S3ABlockOutputStream(S3AFileSystem, String, Progressable, long, S3ADataBlocks$BlockFactory, S3AInstrumentation$OutputStreamStatistics) At S3ABlockOutputStream.java: [line 163]   Exceptional return value of java.io.File.delete() ignored in org.apache.hadoop.fs.s3a.S3ADataBlocks$DiskBlock$FileDeletingInputStream.close() At S3ADataBlocks.java:ignored in org.apache.hadoop.fs.s3a.S3ADataBlocks$DiskBlock$FileDeletingInputStream.close() At S3ADataBlocks.java: [line 787] Subsystem Report/Notes Docker Image:yetus/hadoop:b59b8b7 JIRA Issue HADOOP-13560 GITHUB PR https://github.com/apache/hadoop/pull/130 Optional Tests asflicense compile javac javadoc mvninstall mvnsite unit xml findbugs checkstyle uname Linux 1e87f6c4e839 3.13.0-93-generic #140-Ubuntu SMP Mon Jul 18 21:21:05 UTC 2016 x86_64 x86_64 x86_64 GNU/Linux Build tool maven Personality /testptch/hadoop/patchprocess/precommit/personality/provided.sh git revision branch-2 / 6c98455 Default Java 1.7.0_111 Multi-JDK versions /usr/lib/jvm/java-8-oracle:1.8.0_101 /usr/lib/jvm/java-7-openjdk-amd64:1.7.0_111 findbugs v3.0.0 javac https://builds.apache.org/job/PreCommit-HADOOP-Build/10583/artifact/patchprocess/diff-compile-javac-root-jdk1.8.0_101.txt javac https://builds.apache.org/job/PreCommit-HADOOP-Build/10583/artifact/patchprocess/diff-compile-javac-root-jdk1.7.0_111.txt checkstyle https://builds.apache.org/job/PreCommit-HADOOP-Build/10583/artifact/patchprocess/diff-checkstyle-root.txt whitespace https://builds.apache.org/job/PreCommit-HADOOP-Build/10583/artifact/patchprocess/whitespace-eol.txt whitespace https://builds.apache.org/job/PreCommit-HADOOP-Build/10583/artifact/patchprocess/whitespace-tabs.txt findbugs https://builds.apache.org/job/PreCommit-HADOOP-Build/10583/artifact/patchprocess/new-findbugs-hadoop-tools_hadoop-aws.html JDK v1.7.0_111 Test Results https://builds.apache.org/job/PreCommit-HADOOP-Build/10583/testReport/ modules C: hadoop-common-project/hadoop-common hadoop-tools/hadoop-aws U: . Console output https://builds.apache.org/job/PreCommit-HADOOP-Build/10583/console Powered by Apache Yetus 0.4.0-SNAPSHOT http://yetus.apache.org This message was automatically generated.
          Hide
          stevel@apache.org Steve Loughran added a comment -

          last patch 7129f against s3a ireland with block output enabled on all S3a tests by default

          Failure in distcp

          -------------------------------------------------------------------------------Tests run: 4, Failures: 1, Errors: 0, Skipped: 0, Time elapsed: 110.45 sec <<< FAILURE! - in org.apache.hadoop.fs.contract.s3a.ITestS3AContractDistCplargeFilesFromRemote(org.apache.hadoop.fs.contract.s3a.ITestS3AContractDistCp)  Time elapsed: 38.58 sec  <<< FAILURE! java.lang.AssertionError: wrong length DeprecatedRawLocalFileStatus{path=file:/Users/stevel/Projects/hadoop-trunk/hadoop-tools/hadoop-aws/target/test/data/ITestS3AContractDistCp/largeFilesFromRemote/outputDir/inputDir/file1; isDirectory=false; length=1048576; replication=1; blocksize=33554432; modification_time=1474617816000; access_time=1474617816000; owner=; group=; permission=rw-rw-rw-; isSymlink=false} expected:<11534336> but was:<1048576>
                  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.contract.ContractTestUtils.verifyFileContents(ContractTestUtils.java:202)
                  at org.apache.hadoop.tools.contract.AbstractContractDistCpTest.largeFiles(AbstractContractDistCpTest.java:174)
                  at org.apache.hadoop.tools.contract.AbstractContractDistCpTest.largeFilesFromRemote(AbstractContractDistCpTest.java:108)
                  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.rules.TestWatcher$1.evaluate(TestWatcher.java:55)
                  at org.junit.internal.runners.statements.FailOnTimeout$StatementThread.run(FailOnTimeout.java:74)
          target/failsafe-reports/org.apache.hadoop.fs.contract.s3a.ITestS3AContractDistCp.txt (END)
          

          This is interesting; it is not occuring when run standalone. Race condition or consistency problem surfacing? Or did this patch break upload when block output is enabled everywhere. I may try to add some checks before the discp D/L to verify the source files are of the expected files, so guarantee it's not a consistency.

          Show
          stevel@apache.org Steve Loughran added a comment - last patch 7129f against s3a ireland with block output enabled on all S3a tests by default Failure in distcp -------------------------------------------------------------------------------Tests run: 4, Failures: 1, Errors: 0, Skipped: 0, Time elapsed: 110.45 sec <<< FAILURE! - in org.apache.hadoop.fs.contract.s3a.ITestS3AContractDistCplargeFilesFromRemote(org.apache.hadoop.fs.contract.s3a.ITestS3AContractDistCp) Time elapsed: 38.58 sec <<< FAILURE! java.lang.AssertionError: wrong length DeprecatedRawLocalFileStatus{path=file:/Users/stevel/Projects/hadoop-trunk/hadoop-tools/hadoop-aws/target/test/data/ITestS3AContractDistCp/largeFilesFromRemote/outputDir/inputDir/file1; isDirectory= false ; length=1048576; replication=1; blocksize=33554432; modification_time=1474617816000; access_time=1474617816000; owner=; group=; permission=rw-rw-rw-; isSymlink= false } expected:<11534336> but was:<1048576> 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.contract.ContractTestUtils.verifyFileContents(ContractTestUtils.java:202) at org.apache.hadoop.tools.contract.AbstractContractDistCpTest.largeFiles(AbstractContractDistCpTest.java:174) at org.apache.hadoop.tools.contract.AbstractContractDistCpTest.largeFilesFromRemote(AbstractContractDistCpTest.java:108) 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.rules.TestWatcher$1.evaluate(TestWatcher.java:55) at org.junit.internal.runners.statements.FailOnTimeout$StatementThread.run(FailOnTimeout.java:74) target/failsafe-reports/org.apache.hadoop.fs.contract.s3a.ITestS3AContractDistCp.txt (END) This is interesting; it is not occuring when run standalone. Race condition or consistency problem surfacing? Or did this patch break upload when block output is enabled everywhere. I may try to add some checks before the discp D/L to verify the source files are of the expected files, so guarantee it's not a consistency.
          Hide
          stevel@apache.org Steve Loughran added a comment -

          Commit fc16e03c; Patch 005. Moved all the operations in the block output stream which directly interacted with the s3 client into a new inner class of S3AFilesSystem, WriteOperationState. This cleanly separates interaction between the output stream —buffering of data and queuing of uploads— from the upload process itself. I think S3Guard may be able to do something with this, but I also hope to use it as a start for async directory list/delete operations; this class would track create-time probes, and initiate the async deletion of directory objects after a successful write. That's why there are separate callbacks for writeSuccessful and writeFailed...we will only want to spawn off the deletion when the write succeeded.

          In the process of coding all this, managed to break multipart uploads: this has led to a clearer understanding of how part uploads fail, an improvement in statistics collection and in the test.

          Otherwise,

          • trying to get the imports in sync with branch-2; IDE somehow rearranged things.
          • docs in more detail
          • manual testing through all the FS operations
          • locally switched all the s3a tests into using this (i.e. turned on block output in auth-keys.xml)

          I think this is ready for review and play. I'd recommend the disk block buffer except in the special case that you know that you can upload data faster than you can generate, and you wan't to bypass the disk. But I'd be curious about performance numbers there, especially on distcp operations with s3a as the destination

          Show
          stevel@apache.org Steve Loughran added a comment - Commit fc16e03c; Patch 005. Moved all the operations in the block output stream which directly interacted with the s3 client into a new inner class of S3AFilesSystem, WriteOperationState. This cleanly separates interaction between the output stream —buffering of data and queuing of uploads— from the upload process itself. I think S3Guard may be able to do something with this, but I also hope to use it as a start for async directory list/delete operations; this class would track create-time probes, and initiate the async deletion of directory objects after a successful write. That's why there are separate callbacks for writeSuccessful and writeFailed...we will only want to spawn off the deletion when the write succeeded. In the process of coding all this, managed to break multipart uploads: this has led to a clearer understanding of how part uploads fail, an improvement in statistics collection and in the test. Otherwise, trying to get the imports in sync with branch-2; IDE somehow rearranged things. docs in more detail manual testing through all the FS operations locally switched all the s3a tests into using this (i.e. turned on block output in auth-keys.xml) I think this is ready for review and play. I'd recommend the disk block buffer except in the special case that you know that you can upload data faster than you can generate, and you wan't to bypass the disk. But I'd be curious about performance numbers there, especially on distcp operations with s3a as the destination
          Hide
          stevel@apache.org Steve Loughran added a comment -

          Testing: s3 ireland. One transient failure that I'm taking as a sighting of a eventual consistency mismatch: an object appearing in a HEAD probe after it's deletion.

          testDeleteEmptyDirRecursive(org.apache.hadoop.fs.contract.s3a.ITestS3AContractDelete)  Time elapsed: 1.234 sec  <<< FAILURE!
          java.lang.AssertionError: Deleted file: unexpectedly found s3a://hwdev-steve-ireland/test/testDeleteEmptyDirRecursive as  S3AFileStatus{path=s3a://hwdev-steve-ireland/test/testDeleteEmptyDirRecursive; isDirectory=true; modification_time=0; access_time=0; owner=; group=; permission=rwxrwxrwx; isSymlink=false} isEmptyDirectory=true
          	at org.junit.Assert.fail(Assert.java:88)
          	at org.apache.hadoop.fs.contract.ContractTestUtils.assertPathDoesNotExist(ContractTestUtils.java:714)
          	at org.apache.hadoop.fs.contract.ContractTestUtils.assertDeleted(ContractTestUtils.java:572)
          	at org.apache.hadoop.fs.contract.ContractTestUtils.assertDeleted(ContractTestUtils.java:550)
          	at org.apache.hadoop.fs.contract.AbstractFSContractTestBase.assertDeleted(AbstractFSContractTestBase.java:349)
          	at org.apache.hadoop.fs.contract.AbstractContractDeleteTest.testDeleteEmptyDirRecursive(AbstractContractDeleteTest.java:44)
          	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)
          

          We may want to think about having the base test slightly more consistency aware, that is: retrying if the FS has a blobstore flag set in its contract. For now though, helps collect stats on frequency of this being visible.

          Show
          stevel@apache.org Steve Loughran added a comment - Testing: s3 ireland. One transient failure that I'm taking as a sighting of a eventual consistency mismatch: an object appearing in a HEAD probe after it's deletion. testDeleteEmptyDirRecursive(org.apache.hadoop.fs.contract.s3a.ITestS3AContractDelete) Time elapsed: 1.234 sec <<< FAILURE! java.lang.AssertionError: Deleted file: unexpectedly found s3a: //hwdev-steve-ireland/test/testDeleteEmptyDirRecursive as S3AFileStatus{path=s3a://hwdev-steve-ireland/test/testDeleteEmptyDirRecursive; isDirectory= true ; modification_time=0; access_time=0; owner=; group=; permission=rwxrwxrwx; isSymlink= false } isEmptyDirectory= true at org.junit.Assert.fail(Assert.java:88) at org.apache.hadoop.fs.contract.ContractTestUtils.assertPathDoesNotExist(ContractTestUtils.java:714) at org.apache.hadoop.fs.contract.ContractTestUtils.assertDeleted(ContractTestUtils.java:572) at org.apache.hadoop.fs.contract.ContractTestUtils.assertDeleted(ContractTestUtils.java:550) at org.apache.hadoop.fs.contract.AbstractFSContractTestBase.assertDeleted(AbstractFSContractTestBase.java:349) at org.apache.hadoop.fs.contract.AbstractContractDeleteTest.testDeleteEmptyDirRecursive(AbstractContractDeleteTest.java:44) 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) We may want to think about having the base test slightly more consistency aware, that is: retrying if the FS has a blobstore flag set in its contract. For now though, helps collect stats on frequency of this being visible.
          Hide
          stevel@apache.org Steve Loughran added a comment -

          Patch 006; commit 88e443; scale test runs, with one NPE in logging caught.

          • fix an NPE in a debug log statement for close-with-no-active block;
          • remove unimplemented (and hard to implement meaningfully) bandwidth gauge;
          • diff against branch-2 to reduce delta as much as possible (IDE import changes)

          Lots of work on improving many-GB test runs, including:

          • configurable test timeout in maven
          • preflight validation oftest timeout in big files (and a suggestion of a new timeout size to use); This stops a test being killed after an hour because a timeout was too low in maven or in the test case. Validation assumes 1MB/s bandwidth,
          • bandwidth stats printed on intermediate writes and on upload callbacks, so helping differentiate buffer write and upload speeds, and give someone logging the files something interesting to look at. This becomes quite useful when you are doing a many-GB test and what a bit of a clue what is going on
            2016-09-27 16:17:25,592 [JUnit] INFO  scale.AbstractSTestS3AHugeFiles (AbstractSTestS3AHugeFiles.java:test_010_CreateHugeFile(181)) - [50%] Buffered 10.00 MB out of 20 MB; PUT 0 bytes (8388608 pending) in 1 operations (1 active); elapsedTime=0.86s; write to buffer bandwidth=11.58 MB/s
            2016-09-27 16:17:25,592 [java-sdk-progress-listener-callback-thread] INFO  scale.AbstractSTestS3AHugeFiles (AbstractSTestS3AHugeFiles.java:progressChanged(270)) - Event TRANSFER_PART_STARTED_EVENT, bytes: 0
            2016-09-27 16:17:25,593 [java-sdk-progress-listener-callback-thread] INFO  scale.AbstractSTestS3AHugeFiles (AbstractSTestS3AHugeFiles.java:progressChanged(270)) - Event CLIENT_REQUEST_STARTED_EVENT, bytes: 0
            2016-09-27 16:17:25,593 [java-sdk-progress-listener-callback-thread] INFO  scale.AbstractSTestS3AHugeFiles (AbstractSTestS3AHugeFiles.java:progressChanged(270)) - Event HTTP_REQUEST_STARTED_EVENT, bytes: 0
            2016-09-27 16:17:25,613 [java-sdk-progress-listener-callback-thread] INFO  scale.AbstractSTestS3AHugeFiles (AbstractSTestS3AHugeFiles.java:progressChanged(270)) - Event TRANSFER_PART_STARTED_EVENT, bytes: 0
            2016-09-27 16:17:25,613 [java-sdk-progress-listener-callback-thread] INFO  scale.AbstractSTestS3AHugeFiles (AbstractSTestS3AHugeFiles.java:progressChanged(270)) - Event CLIENT_REQUEST_STARTED_EVENT, bytes: 0
            2016-09-27 16:17:25,614 [java-sdk-progress-listener-callback-thread] INFO  scale.AbstractSTestS3AHugeFiles (AbstractSTestS3AHugeFiles.java:progressChanged(270)) - Event HTTP_REQUEST_STARTED_EVENT, bytes: 0
            2016-09-27 16:17:25,616 [JUnit] INFO  scale.AbstractSTestS3AHugeFiles (AbstractSTestS3AHugeFiles.java:test_010_CreateHugeFile(181)) - [100%] Buffered 20.00 MB out of 20 MB; PUT 0 bytes (16777216 pending) in 2 operations (2 active); elapsedTime=0.89s; write to buffer bandwidth=22.48 MB/s
            2016-09-27 16:17:25,617 [JUnit] INFO  scale.AbstractSTestS3AHugeFiles (AbstractSTestS3AHugeFiles.java:test_010_CreateHugeFile(196)) - Closing file and completing write operation
            2016-09-27 16:17:25,627 [java-sdk-progress-listener-callback-thread] INFO  scale.AbstractSTestS3AHugeFiles (AbstractSTestS3AHugeFiles.java:progressChanged(270)) - Event TRANSFER_PART_STARTED_EVENT, bytes: 0
            2016-09-27 16:17:25,627 [java-sdk-progress-listener-callback-thread] INFO  scale.AbstractSTestS3AHugeFiles (AbstractSTestS3AHugeFiles.java:progressChanged(270)) - Event CLIENT_REQUEST_STARTED_EVENT, bytes: 0
            2016-09-27 16:17:25,627 [java-sdk-progress-listener-callback-thread] INFO  scale.AbstractSTestS3AHugeFiles (AbstractSTestS3AHugeFiles.java:progressChanged(270)) - Event HTTP_REQUEST_STARTED_EVENT, bytes: 0
            2016-09-27 16:17:35,552 [java-sdk-progress-listener-callback-thread] INFO  scale.AbstractSTestS3AHugeFiles (AbstractSTestS3AHugeFiles.java:progressChanged(270)) - Event HTTP_REQUEST_COMPLETED_EVENT, bytes: 0
            2016-09-27 16:17:35,553 [java-sdk-progress-listener-callback-thread] INFO  scale.AbstractSTestS3AHugeFiles (AbstractSTestS3AHugeFiles.java:progressChanged(270)) - Event HTTP_RESPONSE_STARTED_EVENT, bytes: 0
            2016-09-27 16:17:35,553 [java-sdk-progress-listener-callback-thread] INFO  scale.AbstractSTestS3AHugeFiles (AbstractSTestS3AHugeFiles.java:progressChanged(270)) - Event HTTP_RESPONSE_COMPLETED_EVENT, bytes: 0
            2016-09-27 16:17:35,553 [java-sdk-progress-listener-callback-thread] INFO  scale.AbstractSTestS3AHugeFiles (AbstractSTestS3AHugeFiles.java:progressChanged(270)) - Event CLIENT_REQUEST_SUCCESS_EVENT, bytes: 0
            2016-09-27 16:17:35,553 [java-sdk-progress-listener-callback-thread] INFO  scale.AbstractSTestS3AHugeFiles (AbstractSTestS3AHugeFiles.java:progressChanged(260)) - Event TRANSFER_PART_COMPLETED_EVENT, bytes: 0; total uploaded=12 MB in 10.8s; effective upload bandwidth = 1.11 MB/s
            2016-09-27 16:17:41,427 [java-sdk-progress-listener-callback-thread] INFO  scale.AbstractSTestS3AHugeFiles (AbstractSTestS3AHugeFiles.java:progressChanged(270)) - Event HTTP_REQUEST_COMPLETED_EVENT, bytes: 0
            2016-09-27 16:17:41,427 [java-sdk-progress-listener-callback-thread] INFO  scale.AbstractSTestS3AHugeFiles (AbstractSTestS3AHugeFiles.java:progressChanged(270)) - Event HTTP_RESPONSE_STARTED_EVENT, bytes: 0
            2016-09-27 16:17:41,427 [java-sdk-progress-listener-callback-thread] INFO  scale.AbstractSTestS3AHugeFiles (AbstractSTestS3AHugeFiles.java:progressChanged(270)) - Event HTTP_RESPONSE_COMPLETED_EVENT, bytes: 0
            2016-09-27 16:17:41,427 [java-sdk-progress-listener-callback-thread] INFO  scale.AbstractSTestS3AHugeFiles (AbstractSTestS3AHugeFiles.java:progressChanged(270)) - Event CLIENT_REQUEST_SUCCESS_EVENT, bytes: 0
            2016-09-27 16:17:41,428 [java-sdk-progress-listener-callback-thread] INFO  scale.AbstractSTestS3AHugeFiles (AbstractSTestS3AHugeFiles.java:progressChanged(260)) - Event TRANSFER_PART_COMPLETED_EVENT, bytes: 0; total uploaded=20 MB in 16.7s; effective upload bandwidth = 1.20 MB/s
            2016-09-27 16:17:41,438 [java-sdk-progress-listener-callback-thread] INFO  scale.AbstractSTestS3AHugeFiles (AbstractSTestS3AHugeFiles.java:progressChanged(270)) - Event HTTP_REQUEST_COMPLETED_EVENT, bytes: 0
            2016-09-27 16:17:41,438 [java-sdk-progress-listener-callback-thread] INFO  scale.AbstractSTestS3AHugeFiles (AbstractSTestS3AHugeFiles.java:progressChanged(270)) - Event HTTP_RESPONSE_STARTED_EVENT, bytes: 0
            2016-09-27 16:17:41,438 [java-sdk-progress-listener-callback-thread] INFO  scale.AbstractSTestS3AHugeFiles (AbstractSTestS3AHugeFiles.java:progressChanged(270)) - Event HTTP_RESPONSE_COMPLETED_EVENT, bytes: 0
            2016-09-27 16:17:41,438 [java-sdk-progress-listener-callback-thread] INFO  scale.AbstractSTestS3AHugeFiles (AbstractSTestS3AHugeFiles.java:progressChanged(270)) - Event CLIENT_REQUEST_SUCCESS_EVENT, bytes: 0
            2016-09-27 16:17:41,439 [java-sdk-progress-listener-callback-thread] INFO  scale.AbstractSTestS3AHugeFiles (AbstractSTestS3AHugeFiles.java:progressChanged(260)) - Event TRANSFER_PART_COMPLETED_EVENT, bytes: 0; total uploaded=20 MB in 16.7s; effective upload bandwidth = 1.20 MB/s
            2016-09-27 16:17:41,919 [JUnit] INFO  contract.ContractTestUtils (ContractTestUtils.java:end(1365)) - Duration of time to close() output stream: 16,302,440,012 nS
            2016-09-27 16:17:41,919 [JUnit] INFO  contract.ContractTestUtils (ContractTestUtils.java:end(1365)) - Duration of time to write 20 MB in blocks of 65536: 17,193,443,672 nS
            2016-09-27 16:17:41,920 [JUnit] INFO  scale.AbstractSTestS3AHugeFiles (AbstractSTestS3AHugeFiles.java:test_010_CreateHugeFile(206)) - Time per MB to write = 859,672,183 nS
            2016-09-27 16:17:41,921 [JUnit] INFO  contract.ContractTestUtils (ContractTestUtils.java:bandwidth(917)) - Bandwidth = 1.163234  MB/S
            2016-09-27 16:17:41,921 [JUnit] INFO  scale.AbstractSTestS3AHugeFiles (AbstractSTestS3AHugeFiles.java:test_010_CreateHugeFile(211)) - PUT 20971520 bytes in 3 operations; 6 MB/operation
            2016-09-27 16:17:41,921 [JUnit] INFO  scale.AbstractSTestS3AHugeFiles (AbstractSTestS3AHugeFiles.java:test_010_CreateHugeFile(214)) - Time per PUT 5,731,147,890 nS
            2016-09-27 16:17:42,303 [JUnit] INFO  scale.S3AScaleTestBase (S3AScaleTestBase.java:describe(161)) -
          

          Tests: s3 ireland; using test files of 4GB. Saw a couple of socket timeouts reading the data back in; S3a had logged and retried. Created HADOOP-13664 to consider using a Hadoop RetryPolicy here.

          Show
          stevel@apache.org Steve Loughran added a comment - Patch 006; commit 88e443; scale test runs, with one NPE in logging caught. fix an NPE in a debug log statement for close-with-no-active block; remove unimplemented (and hard to implement meaningfully) bandwidth gauge; diff against branch-2 to reduce delta as much as possible (IDE import changes) Lots of work on improving many-GB test runs, including: configurable test timeout in maven preflight validation oftest timeout in big files (and a suggestion of a new timeout size to use); This stops a test being killed after an hour because a timeout was too low in maven or in the test case. Validation assumes 1MB/s bandwidth, bandwidth stats printed on intermediate writes and on upload callbacks, so helping differentiate buffer write and upload speeds, and give someone logging the files something interesting to look at. This becomes quite useful when you are doing a many-GB test and what a bit of a clue what is going on 2016-09-27 16:17:25,592 [JUnit] INFO scale.AbstractSTestS3AHugeFiles (AbstractSTestS3AHugeFiles.java:test_010_CreateHugeFile(181)) - [50%] Buffered 10.00 MB out of 20 MB; PUT 0 bytes (8388608 pending) in 1 operations (1 active); elapsedTime=0.86s; write to buffer bandwidth=11.58 MB/s 2016-09-27 16:17:25,592 [java-sdk-progress-listener-callback-thread] INFO scale.AbstractSTestS3AHugeFiles (AbstractSTestS3AHugeFiles.java:progressChanged(270)) - Event TRANSFER_PART_STARTED_EVENT, bytes: 0 2016-09-27 16:17:25,593 [java-sdk-progress-listener-callback-thread] INFO scale.AbstractSTestS3AHugeFiles (AbstractSTestS3AHugeFiles.java:progressChanged(270)) - Event CLIENT_REQUEST_STARTED_EVENT, bytes: 0 2016-09-27 16:17:25,593 [java-sdk-progress-listener-callback-thread] INFO scale.AbstractSTestS3AHugeFiles (AbstractSTestS3AHugeFiles.java:progressChanged(270)) - Event HTTP_REQUEST_STARTED_EVENT, bytes: 0 2016-09-27 16:17:25,613 [java-sdk-progress-listener-callback-thread] INFO scale.AbstractSTestS3AHugeFiles (AbstractSTestS3AHugeFiles.java:progressChanged(270)) - Event TRANSFER_PART_STARTED_EVENT, bytes: 0 2016-09-27 16:17:25,613 [java-sdk-progress-listener-callback-thread] INFO scale.AbstractSTestS3AHugeFiles (AbstractSTestS3AHugeFiles.java:progressChanged(270)) - Event CLIENT_REQUEST_STARTED_EVENT, bytes: 0 2016-09-27 16:17:25,614 [java-sdk-progress-listener-callback-thread] INFO scale.AbstractSTestS3AHugeFiles (AbstractSTestS3AHugeFiles.java:progressChanged(270)) - Event HTTP_REQUEST_STARTED_EVENT, bytes: 0 2016-09-27 16:17:25,616 [JUnit] INFO scale.AbstractSTestS3AHugeFiles (AbstractSTestS3AHugeFiles.java:test_010_CreateHugeFile(181)) - [100%] Buffered 20.00 MB out of 20 MB; PUT 0 bytes (16777216 pending) in 2 operations (2 active); elapsedTime=0.89s; write to buffer bandwidth=22.48 MB/s 2016-09-27 16:17:25,617 [JUnit] INFO scale.AbstractSTestS3AHugeFiles (AbstractSTestS3AHugeFiles.java:test_010_CreateHugeFile(196)) - Closing file and completing write operation 2016-09-27 16:17:25,627 [java-sdk-progress-listener-callback-thread] INFO scale.AbstractSTestS3AHugeFiles (AbstractSTestS3AHugeFiles.java:progressChanged(270)) - Event TRANSFER_PART_STARTED_EVENT, bytes: 0 2016-09-27 16:17:25,627 [java-sdk-progress-listener-callback-thread] INFO scale.AbstractSTestS3AHugeFiles (AbstractSTestS3AHugeFiles.java:progressChanged(270)) - Event CLIENT_REQUEST_STARTED_EVENT, bytes: 0 2016-09-27 16:17:25,627 [java-sdk-progress-listener-callback-thread] INFO scale.AbstractSTestS3AHugeFiles (AbstractSTestS3AHugeFiles.java:progressChanged(270)) - Event HTTP_REQUEST_STARTED_EVENT, bytes: 0 2016-09-27 16:17:35,552 [java-sdk-progress-listener-callback-thread] INFO scale.AbstractSTestS3AHugeFiles (AbstractSTestS3AHugeFiles.java:progressChanged(270)) - Event HTTP_REQUEST_COMPLETED_EVENT, bytes: 0 2016-09-27 16:17:35,553 [java-sdk-progress-listener-callback-thread] INFO scale.AbstractSTestS3AHugeFiles (AbstractSTestS3AHugeFiles.java:progressChanged(270)) - Event HTTP_RESPONSE_STARTED_EVENT, bytes: 0 2016-09-27 16:17:35,553 [java-sdk-progress-listener-callback-thread] INFO scale.AbstractSTestS3AHugeFiles (AbstractSTestS3AHugeFiles.java:progressChanged(270)) - Event HTTP_RESPONSE_COMPLETED_EVENT, bytes: 0 2016-09-27 16:17:35,553 [java-sdk-progress-listener-callback-thread] INFO scale.AbstractSTestS3AHugeFiles (AbstractSTestS3AHugeFiles.java:progressChanged(270)) - Event CLIENT_REQUEST_SUCCESS_EVENT, bytes: 0 2016-09-27 16:17:35,553 [java-sdk-progress-listener-callback-thread] INFO scale.AbstractSTestS3AHugeFiles (AbstractSTestS3AHugeFiles.java:progressChanged(260)) - Event TRANSFER_PART_COMPLETED_EVENT, bytes: 0; total uploaded=12 MB in 10.8s; effective upload bandwidth = 1.11 MB/s 2016-09-27 16:17:41,427 [java-sdk-progress-listener-callback-thread] INFO scale.AbstractSTestS3AHugeFiles (AbstractSTestS3AHugeFiles.java:progressChanged(270)) - Event HTTP_REQUEST_COMPLETED_EVENT, bytes: 0 2016-09-27 16:17:41,427 [java-sdk-progress-listener-callback-thread] INFO scale.AbstractSTestS3AHugeFiles (AbstractSTestS3AHugeFiles.java:progressChanged(270)) - Event HTTP_RESPONSE_STARTED_EVENT, bytes: 0 2016-09-27 16:17:41,427 [java-sdk-progress-listener-callback-thread] INFO scale.AbstractSTestS3AHugeFiles (AbstractSTestS3AHugeFiles.java:progressChanged(270)) - Event HTTP_RESPONSE_COMPLETED_EVENT, bytes: 0 2016-09-27 16:17:41,427 [java-sdk-progress-listener-callback-thread] INFO scale.AbstractSTestS3AHugeFiles (AbstractSTestS3AHugeFiles.java:progressChanged(270)) - Event CLIENT_REQUEST_SUCCESS_EVENT, bytes: 0 2016-09-27 16:17:41,428 [java-sdk-progress-listener-callback-thread] INFO scale.AbstractSTestS3AHugeFiles (AbstractSTestS3AHugeFiles.java:progressChanged(260)) - Event TRANSFER_PART_COMPLETED_EVENT, bytes: 0; total uploaded=20 MB in 16.7s; effective upload bandwidth = 1.20 MB/s 2016-09-27 16:17:41,438 [java-sdk-progress-listener-callback-thread] INFO scale.AbstractSTestS3AHugeFiles (AbstractSTestS3AHugeFiles.java:progressChanged(270)) - Event HTTP_REQUEST_COMPLETED_EVENT, bytes: 0 2016-09-27 16:17:41,438 [java-sdk-progress-listener-callback-thread] INFO scale.AbstractSTestS3AHugeFiles (AbstractSTestS3AHugeFiles.java:progressChanged(270)) - Event HTTP_RESPONSE_STARTED_EVENT, bytes: 0 2016-09-27 16:17:41,438 [java-sdk-progress-listener-callback-thread] INFO scale.AbstractSTestS3AHugeFiles (AbstractSTestS3AHugeFiles.java:progressChanged(270)) - Event HTTP_RESPONSE_COMPLETED_EVENT, bytes: 0 2016-09-27 16:17:41,438 [java-sdk-progress-listener-callback-thread] INFO scale.AbstractSTestS3AHugeFiles (AbstractSTestS3AHugeFiles.java:progressChanged(270)) - Event CLIENT_REQUEST_SUCCESS_EVENT, bytes: 0 2016-09-27 16:17:41,439 [java-sdk-progress-listener-callback-thread] INFO scale.AbstractSTestS3AHugeFiles (AbstractSTestS3AHugeFiles.java:progressChanged(260)) - Event TRANSFER_PART_COMPLETED_EVENT, bytes: 0; total uploaded=20 MB in 16.7s; effective upload bandwidth = 1.20 MB/s 2016-09-27 16:17:41,919 [JUnit] INFO contract.ContractTestUtils (ContractTestUtils.java:end(1365)) - Duration of time to close() output stream: 16,302,440,012 nS 2016-09-27 16:17:41,919 [JUnit] INFO contract.ContractTestUtils (ContractTestUtils.java:end(1365)) - Duration of time to write 20 MB in blocks of 65536: 17,193,443,672 nS 2016-09-27 16:17:41,920 [JUnit] INFO scale.AbstractSTestS3AHugeFiles (AbstractSTestS3AHugeFiles.java:test_010_CreateHugeFile(206)) - Time per MB to write = 859,672,183 nS 2016-09-27 16:17:41,921 [JUnit] INFO contract.ContractTestUtils (ContractTestUtils.java:bandwidth(917)) - Bandwidth = 1.163234 MB/S 2016-09-27 16:17:41,921 [JUnit] INFO scale.AbstractSTestS3AHugeFiles (AbstractSTestS3AHugeFiles.java:test_010_CreateHugeFile(211)) - PUT 20971520 bytes in 3 operations; 6 MB/operation 2016-09-27 16:17:41,921 [JUnit] INFO scale.AbstractSTestS3AHugeFiles (AbstractSTestS3AHugeFiles.java:test_010_CreateHugeFile(214)) - Time per PUT 5,731,147,890 nS 2016-09-27 16:17:42,303 [JUnit] INFO scale.S3AScaleTestBase (S3AScaleTestBase.java:describe(161)) - Tests: s3 ireland; using test files of 4GB. Saw a couple of socket timeouts reading the data back in; S3a had logged and retried. Created HADOOP-13664 to consider using a Hadoop RetryPolicy here.
          Hide
          stevel@apache.org Steve Loughran added a comment -

          BTW, that 1.2MB/s is because I'm doing the tests remotely over a FTTC connection. On renaming files I've seen ~6MB/s during the rename, which is going to be a metric of disk IO rates, as that's the in-S3 copy operation

          Show
          stevel@apache.org Steve Loughran added a comment - BTW, that 1.2MB/s is because I'm doing the tests remotely over a FTTC connection. On renaming files I've seen ~6MB/s during the rename, which is going to be a metric of disk IO rates, as that's the in-S3 copy operation
          Hide
          cnauroth Chris Nauroth added a comment -

          Thank you, Steve. I have started reviewing patch revision 006. I haven't read through all of it yet, but here is my feedback so far.

          This patch does not apply to current trunk, so we'll eventually need a different patch for trunk.

          All access to S3ABlockOutputStream#closed happens through synchronized methods. Would it be simpler to change the data type to straight boolean, or do you prefer to stick with AtomicBoolean?

          S3ABlockOutputStream#now returns time in milliseconds, but the JavaDocs state nanoseconds. Did you want System#nanoTime or possibly org.apache.hadoop.util.Time#monotonicNow for a millisecond measurement that's safe against system clock changes?

          Can ITestS3AHuge* be made to run in parallel instead of sequential? It appears these tests are already sufficiently isolated from one another. They call S3AScaleTestBase#getTestPath, so they are guaranteed to operate on isolated paths within the bucket. They also disable the multi-part upload purge in S3AScaleTestBase#setUp. Is there another isolation problem I missed, or is the idea more that you don't want activity from another test running in parallel to pollute metrics reported from the scale tests due to bandwidth limitations or throttling?

          Show
          cnauroth Chris Nauroth added a comment - Thank you, Steve. I have started reviewing patch revision 006. I haven't read through all of it yet, but here is my feedback so far. This patch does not apply to current trunk, so we'll eventually need a different patch for trunk. All access to S3ABlockOutputStream#closed happens through synchronized methods. Would it be simpler to change the data type to straight boolean , or do you prefer to stick with AtomicBoolean ? S3ABlockOutputStream#now returns time in milliseconds, but the JavaDocs state nanoseconds. Did you want System#nanoTime or possibly org.apache.hadoop.util.Time#monotonicNow for a millisecond measurement that's safe against system clock changes? Can ITestS3AHuge* be made to run in parallel instead of sequential? It appears these tests are already sufficiently isolated from one another. They call S3AScaleTestBase#getTestPath , so they are guaranteed to operate on isolated paths within the bucket. They also disable the multi-part upload purge in S3AScaleTestBase#setUp . Is there another isolation problem I missed, or is the idea more that you don't want activity from another test running in parallel to pollute metrics reported from the scale tests due to bandwidth limitations or throttling?
          Hide
          stevel@apache.org Steve Loughran added a comment -

          so we'll eventually need a different patch for trunk.

          no problem

          All access to S3ABlockOutputStream#closed happens through synchronized methods.

          regarding close, i'd actually planned to make close() unsynced; clearly I didn't do that final step. By going to unsync we avoid deadlocks if there's >1 close() call made and the first one is waiting for the upload to complete. On that topic, should we add something about close() of fs/stream not being blocking to the FS spec?

          S3ABlockOutputStream#now returns time in milliseconds, but the JavaDocs state nanoseconds.

          well spotted. It's only being used for some metrics about time for blocks to get through queue/uploaded. Changed the javadocs

          Can ITestS3AHuge* be made to run in parallel instead of sequential?

          the problem here is that the tests saturate the entire network. If you run them in parallel with smaller tests, everything slows down. If you run them in parallel with each other, things come to a complete halt as the bandwidth is split across the tests. It doesn't get any faster, no matter how many cores you have. This is network link bound. It's why I added the new scale profile; these really are "set them running and go away" test runs, batch jobs rather than things you would do every iteration, at least unless you were actually playing with the output streams.

          I could see a single huge file test being scheduled while all the small tests runs, as long as they don't go near the multipart purge settings. What I can't see in failsafe or other junit test runner the way to specify "run the huge tests sequentially while running the other tests in parallel".

          regarding the multipart purge, that was causing problems in parallel test runs because the purge time was being set such that a large enough upload would fail, some of the interim parts would have already been purged by the time the commit got through. This isn't a problem with small files, but once you go into the many-GB you start to hit various test run scale issues (generally: timeouts of purge, failsafe, JUnit test timeout), and encounter transient network failures. That's why there's some retry logic on the multipart commit phase: I encountered it.

          Show
          stevel@apache.org Steve Loughran added a comment - so we'll eventually need a different patch for trunk. no problem All access to S3ABlockOutputStream#closed happens through synchronized methods. regarding close, i'd actually planned to make close() unsynced; clearly I didn't do that final step. By going to unsync we avoid deadlocks if there's >1 close() call made and the first one is waiting for the upload to complete. On that topic, should we add something about close() of fs/stream not being blocking to the FS spec? S3ABlockOutputStream#now returns time in milliseconds, but the JavaDocs state nanoseconds. well spotted. It's only being used for some metrics about time for blocks to get through queue/uploaded. Changed the javadocs Can ITestS3AHuge* be made to run in parallel instead of sequential? the problem here is that the tests saturate the entire network. If you run them in parallel with smaller tests, everything slows down. If you run them in parallel with each other, things come to a complete halt as the bandwidth is split across the tests. It doesn't get any faster, no matter how many cores you have. This is network link bound. It's why I added the new scale profile; these really are "set them running and go away" test runs, batch jobs rather than things you would do every iteration, at least unless you were actually playing with the output streams. I could see a single huge file test being scheduled while all the small tests runs, as long as they don't go near the multipart purge settings. What I can't see in failsafe or other junit test runner the way to specify "run the huge tests sequentially while running the other tests in parallel". regarding the multipart purge, that was causing problems in parallel test runs because the purge time was being set such that a large enough upload would fail, some of the interim parts would have already been purged by the time the commit got through. This isn't a problem with small files, but once you go into the many-GB you start to hit various test run scale issues (generally: timeouts of purge, failsafe, JUnit test timeout), and encounter transient network failures. That's why there's some retry logic on the multipart commit phase: I encountered it.
          Hide
          cnauroth Chris Nauroth added a comment -

          At this point, I've read through all of patch 006 except for the documentation updates. Very nice patch! Here is more feedback on the code, and I'll start reviewing the documentation changes next.

          I see the checks in S3ADataBlocks#validateWriteArgs are taken from java.io.OutputStream#write. Would you please add a note in the JavaDoc that this is implemented specifically to match that contract?

          It seems the validation check in DataBlock#verifyState would only ever throw an exception if there was a bug in our S3A code's state transition logic, not a recoverable situation for the client. If so, then is a runtime exception more appropriate, such as IllegalStateException? If so, then that would also avoid the ignored exception in DataBlock#enterClosedState.

          The JavaDoc for DataBlock#flush states that it is only valid in writing state. Does it make sense to move the validation check for that from DiskBlock#flush up into the base class?

          In ByteArrayBlock#startUpload, can the call to buffer.reset() be removed? Resetting sets an internal counter without releasing any of the underlying heap space, which is what the buffer = null immediately afterwards achieves.

          In the single-byte ByteBufferInputStream#read, I think checking hasRemaining() would be more readable than handling BufferUnderflowException, and it would look consistent with similar logic in the multi-byte read.

          Please mark final on the member fields of ByteBufferBlock and DiskBlock where appropriate.

                try {
                  out.flush();
                  out.close();
                } finally {
                  out.close();
                  out = null;
                }
          

          I was a bit confused by the above code in DiskBlock#startUpload. I think what this is trying to do (and please correct me if I'm wrong) is to make sure we propagate an exception thrown from flush. Therefore, we can't simplify this code to just a single close call, because even though BufferedOutputStream#close automatically flushes, it ignores exceptions from flush, and we'd lose those error details. If this is the intent, would you please add a comment explaining it?

          Should DiskBlock#startUpload wrap the returned stream in a BufferedInputStream?

          Is ForwardingInputStream necessary? It looks like FilterInputStream, so can FileDeletingInputStream subclass that instead?

          In MultiPartUpload#shouldRetry, is Thread.interrupted() supposed to be Thread.currentThread().interrupt()? If InterruptedException has been thrown, then the interrupted flag has been cleared already, so I think Thread.interrupted() would effectively be a no-op.

          Can we avoid creating the LocalDirAllocator when we are not using S3AOutputStream or DiskBlockFactory?

          I'm unclear why MAX_THREADS was downtuned to 1 in ITestS3ABlockingThreadPool. If that's intentional, please also update the class-level comment that states 2 threads and describe why the test is still sufficient with only 1 thread.

          Show
          cnauroth Chris Nauroth added a comment - At this point, I've read through all of patch 006 except for the documentation updates. Very nice patch! Here is more feedback on the code, and I'll start reviewing the documentation changes next. I see the checks in S3ADataBlocks#validateWriteArgs are taken from java.io.OutputStream#write . Would you please add a note in the JavaDoc that this is implemented specifically to match that contract? It seems the validation check in DataBlock#verifyState would only ever throw an exception if there was a bug in our S3A code's state transition logic, not a recoverable situation for the client. If so, then is a runtime exception more appropriate, such as IllegalStateException ? If so, then that would also avoid the ignored exception in DataBlock#enterClosedState . The JavaDoc for DataBlock#flush states that it is only valid in writing state. Does it make sense to move the validation check for that from DiskBlock#flush up into the base class? In ByteArrayBlock#startUpload , can the call to buffer.reset() be removed? Resetting sets an internal counter without releasing any of the underlying heap space, which is what the buffer = null immediately afterwards achieves. In the single-byte ByteBufferInputStream#read , I think checking hasRemaining() would be more readable than handling BufferUnderflowException , and it would look consistent with similar logic in the multi-byte read . Please mark final on the member fields of ByteBufferBlock and DiskBlock where appropriate. try { out.flush(); out.close(); } finally { out.close(); out = null ; } I was a bit confused by the above code in DiskBlock#startUpload . I think what this is trying to do (and please correct me if I'm wrong) is to make sure we propagate an exception thrown from flush . Therefore, we can't simplify this code to just a single close call, because even though BufferedOutputStream#close automatically flushes, it ignores exceptions from flush, and we'd lose those error details. If this is the intent, would you please add a comment explaining it? Should DiskBlock#startUpload wrap the returned stream in a BufferedInputStream ? Is ForwardingInputStream necessary? It looks like FilterInputStream , so can FileDeletingInputStream subclass that instead? In MultiPartUpload#shouldRetry , is Thread.interrupted() supposed to be Thread.currentThread().interrupt() ? If InterruptedException has been thrown, then the interrupted flag has been cleared already, so I think Thread.interrupted() would effectively be a no-op. Can we avoid creating the LocalDirAllocator when we are not using S3AOutputStream or DiskBlockFactory ? I'm unclear why MAX_THREADS was downtuned to 1 in ITestS3ABlockingThreadPool . If that's intentional, please also update the class-level comment that states 2 threads and describe why the test is still sufficient with only 1 thread.
          Hide
          cnauroth Chris Nauroth added a comment -

          The documentation changes are looking good too. The content is fine, and I just have minor comments.

          The new troubleshooting sections are not nested correctly under the "Troubleshooting S3A" top-level topic.

          blocks of data (in S3 terms, "parts of a multipart upload). This can make more
          

          Unmatched double-quotes.

          to local hard disks in prior to upload. This minimizes the amount of memory
          

          Remove "in".

          This makes them a foundational part of the benchmarking
          

          Missing period.

          mvn verify  -Pparallel-tests,scale -DtestsThreadCount=8 
          

          As per discussion on HADOOP-13614, we effectively can't use -P because of Maven limitations. I suggest adding property-based activation to the new profile, so we can activate scale tests by passing -Dscale.

          <property>
            <name>fs.s3a.scale.test.huge.filesize</name>
            <value>20</value>
          </property>
          

          I suggest that we use Configuration#getLongBytes for this to support configuring this with more human-readable values that include the unit explicitly. I probably ought to go back and do the same for scale.test.distcp.file.size.kb.

          Show
          cnauroth Chris Nauroth added a comment - The documentation changes are looking good too. The content is fine, and I just have minor comments. The new troubleshooting sections are not nested correctly under the "Troubleshooting S3A" top-level topic. blocks of data (in S3 terms, "parts of a multipart upload). This can make more Unmatched double-quotes. to local hard disks in prior to upload. This minimizes the amount of memory Remove "in". This makes them a foundational part of the benchmarking Missing period. mvn verify -Pparallel-tests,scale -DtestsThreadCount=8 As per discussion on HADOOP-13614 , we effectively can't use -P because of Maven limitations. I suggest adding property-based activation to the new profile, so we can activate scale tests by passing -Dscale . <property> <name>fs.s3a.scale.test.huge.filesize</name> <value>20</value> </property> I suggest that we use Configuration#getLongBytes for this to support configuring this with more human-readable values that include the unit explicitly. I probably ought to go back and do the same for scale.test.distcp.file.size.kb.
          Hide
          cnauroth Chris Nauroth added a comment -

          I just realized that these comments would have been better on the GitHub pull request. Next time...

          Show
          cnauroth Chris Nauroth added a comment - I just realized that these comments would have been better on the GitHub pull request. Next time...
          Hide
          stevel@apache.org Steve Loughran added a comment -

          no , they're fine here

          Show
          stevel@apache.org Steve Loughran added a comment - no , they're fine here
          Hide
          rajesh.balamohan Rajesh Balamohan added a comment -

          S3ABlockOutputStream::initiateMultiPartUpload() has the following

          LOG.debug("Initiating Multipart upload for block {}", currentBlock);
          

          In S3ADataBlocks.java, patch has the following for ByteArrayBlock

          @Override
              public String toString() {
                return "ByteArrayBlock{" +
                    "state=" + getState() +
                    ", buffer=" + buffer +
                    ", limit=" + limit +
                    ", dataSize=" + dataSize +
                    '}';
              }
          

          When DEBUG log was enabled to check the AWS traffic, it ended up printing the entire contents of the buffer. When trying to debug large data transfer (4 GB in my case), it ended up printing huge chunks which may not be needed. Would it be possible to only the buffer sizes?.

          Show
          rajesh.balamohan Rajesh Balamohan added a comment - S3ABlockOutputStream::initiateMultiPartUpload() has the following LOG.debug("Initiating Multipart upload for block {}", currentBlock); In S3ADataBlocks.java, patch has the following for ByteArrayBlock @Override public String toString() { return "ByteArrayBlock{" + "state=" + getState() + ", buffer=" + buffer + ", limit=" + limit + ", dataSize=" + dataSize + '}'; } When DEBUG log was enabled to check the AWS traffic, it ended up printing the entire contents of the buffer. When trying to debug large data transfer (4 GB in my case), it ended up printing huge chunks which may not be needed. Would it be possible to only the buffer sizes?.
          Hide
          stevel@apache.org Steve Loughran added a comment -

          wow. I'd better fix that

          Show
          stevel@apache.org Steve Loughran added a comment - wow. I'd better fix that
          Hide
          stevel@apache.org Steve Loughran added a comment -

          latest PR addresses chris's and rajesh's comments

          Show
          stevel@apache.org Steve Loughran added a comment - latest PR addresses chris's and rajesh's comments
          Hide
          stevel@apache.org Steve Loughran added a comment -

          HADOOP-13566 highlights that S3AFastOutputStream NPEd on a write of a closed stream. Make sure there is a test here for the same action.

          Show
          stevel@apache.org Steve Loughran added a comment - HADOOP-13566 highlights that S3AFastOutputStream NPEd on a write of a closed stream. Make sure there is a test here for the same action.
          Hide
          stevel@apache.org Steve Loughran added a comment -

          think I've addressed most of them. Realised that commenting in github makes replying to them one by one easily.

          anyway, to most of the anwers are "yes, you are write, fixed"

          one exception, why {{ should DiskBlock#startUpload wrap the returned stream in a BufferedInputStream)}}.

          no, because if there is a transient PUT failure and AWS tries to replay the PUT, it can't roll back the buffered input stream back to the beginning of the file.; only back the beginning of the last buffered area. This is something that surfaced in the huge file tests.

          Show
          stevel@apache.org Steve Loughran added a comment - think I've addressed most of them. Realised that commenting in github makes replying to them one by one easily. anyway, to most of the anwers are "yes, you are write, fixed" one exception, why {{ should DiskBlock#startUpload wrap the returned stream in a BufferedInputStream)}}. no, because if there is a transient PUT failure and AWS tries to replay the PUT, it can't roll back the buffered input stream back to the beginning of the file.; only back the beginning of the last buffered area. This is something that surfaced in the huge file tests.
          Hide
          stevel@apache.org Steve Loughran added a comment -

          Latest patch: factored out semaphore queue in front of an ExecutorService into SemaphoredDelegatingExecutor. As well as being used to manage thread pool load, each block output stream gets its own submissions to the central thread pool limited by the property fs.s3a.block.output.active.limit. This will let us have a larger common pool, but still limit the amount of bandwidth a single stream can consume. Which I need as I want to use that same thread pool for parallel rename operations and async mkdir/delete dir operations.

          Show
          stevel@apache.org Steve Loughran added a comment - Latest patch: factored out semaphore queue in front of an ExecutorService into SemaphoredDelegatingExecutor . As well as being used to manage thread pool load, each block output stream gets its own submissions to the central thread pool limited by the property fs.s3a.block.output.active.limit . This will let us have a larger common pool, but still limit the amount of bandwidth a single stream can consume. Which I need as I want to use that same thread pool for parallel rename operations and async mkdir/delete dir operations.
          Hide
          stevel@apache.org Steve Loughran added a comment -

          tested: s3a ireland

          Show
          stevel@apache.org Steve Loughran added a comment - tested: s3a ireland
          Hide
          stevel@apache.org Steve Loughran added a comment -

          Updated patch; tested s3a ireland

          Uses fs.s3a.fast.upload as the switch to enable the new block output stream. This makes it consistent with its (now deleted) predecessor; that is un-deprecated, with all configuration options changed to use fast.upload in their names; FAST_UPLOAD in their fieldnames.

          I've tried to document all this, and add a new section on tuning queue sizes.

          Show
          stevel@apache.org Steve Loughran added a comment - Updated patch; tested s3a ireland Uses fs.s3a.fast.upload as the switch to enable the new block output stream. This makes it consistent with its (now deleted) predecessor; that is un-deprecated, with all configuration options changed to use fast.upload in their names; FAST_UPLOAD in their fieldnames. I've tried to document all this, and add a new section on tuning queue sizes.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user cnauroth commented on a diff in the pull request:

          https://github.com/apache/hadoop/pull/130#discussion_r82090027

          — Diff: hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md —
          @@ -1250,6 +1569,144 @@ can be used:
          Using the explicit endpoint for the region is recommended for speed and the
          ability to use the V4 signing API.

          +
          +## "Timeout waiting for connection from pool" when writing to S3A
          — End diff –

          I tried an `mvn site` build, and it looks like the new troubleshooting sections still aren't nested correctly. I believe it should be `###` instead of `##`.

          Show
          githubbot ASF GitHub Bot added a comment - Github user cnauroth commented on a diff in the pull request: https://github.com/apache/hadoop/pull/130#discussion_r82090027 — Diff: hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md — @@ -1250,6 +1569,144 @@ can be used: Using the explicit endpoint for the region is recommended for speed and the ability to use the V4 signing API. + +## "Timeout waiting for connection from pool" when writing to S3A — End diff – I tried an `mvn site` build, and it looks like the new troubleshooting sections still aren't nested correctly. I believe it should be `###` instead of `##`.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user cnauroth commented on a diff in the pull request:

          https://github.com/apache/hadoop/pull/130#discussion_r82087028

          — Diff: hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ADataBlocks.java —
          @@ -0,0 +1,816 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +
          +package org.apache.hadoop.fs.s3a;
          +
          +import java.io.BufferedOutputStream;
          +import java.io.ByteArrayInputStream;
          +import java.io.ByteArrayOutputStream;
          +import java.io.Closeable;
          +import java.io.EOFException;
          +import java.io.File;
          +import java.io.FileInputStream;
          +import java.io.FileNotFoundException;
          +import java.io.FileOutputStream;
          +import java.io.FilterInputStream;
          +import java.io.IOException;
          +import java.io.InputStream;
          +import java.nio.BufferUnderflowException;
          +import java.nio.ByteBuffer;
          +import java.util.concurrent.atomic.AtomicBoolean;
          +import java.util.concurrent.atomic.AtomicInteger;
          +
          +import com.google.common.base.Preconditions;
          +import org.slf4j.Logger;
          +import org.slf4j.LoggerFactory;
          +
          +import org.apache.hadoop.fs.FSExceptionMessages;
          +import org.apache.hadoop.util.DirectBufferPool;
          +
          +import static org.apache.hadoop.fs.s3a.S3ADataBlocks.DataBlock.DestState.*;
          +
          +/**
          + * Set of classes to support output streaming into blocks which are then
          + * uploaded as partitions.
          + */
          +final class S3ADataBlocks {
          +
          + private static final Logger LOG = LoggerFactory.getLogger(S3ADataBlocks.class);
          +
          + private S3ADataBlocks()

          { + }

          +
          + /**
          + * Validate args to a write command. These are the same validation checks
          + * expected for any implementation of

          {@code OutputStream.write()}

          .
          + * @param b byte array containing data
          + * @param off offset in array where to start
          + * @param len number of bytes to be written
          + * @throws NullPointerException for a null buffer
          + * @throws IndexOutOfBoundsException if indices are out of range
          + */
          + static void validateWriteArgs(byte[] b, int off, int len)
          + throws IOException {
          + Preconditions.checkNotNull(b);
          + if ((off < 0) || (off > b.length) || (len < 0) ||
          + ((off + len) > b.length) || ((off + len) < 0))

          { + throw new IndexOutOfBoundsException( + "write (b[" + b.length + "], " + off + ", " + len + ')'); + }

          + }
          +
          + /**
          + * Create a factory.
          + * @param owner factory owner
          + * @param name factory name -the option from

          {@link Constants}

          .
          + * @return the factory, ready to be initialized.
          + * @throws IllegalArgumentException if the name is unknown.
          + */
          + static BlockFactory createFactory(S3AFileSystem owner,
          + String name) {
          + switch (name)

          { + case Constants.BLOCK_OUTPUT_BUFFER_ARRAY: + return new ArrayBlockFactory(owner); + case Constants.BLOCK_OUTPUT_BUFFER_DISK: + return new DiskBlockFactory(owner); + case Constants.BLOCK_OUTPUT_BYTEBUFFER: + return new ByteBufferBlockFactory(owner); + default: + throw new IllegalArgumentException("Unsupported block buffer" + + " \"" + name + '"'); + }

          + }
          +
          + /**
          + * Base class for block factories.
          + */
          + static abstract class BlockFactory implements Closeable {
          +
          + /**
          + * Owner.
          + */
          + protected final S3AFileSystem owner;
          +
          + protected BlockFactory(S3AFileSystem owner)

          { + this.owner = owner; + }

          +
          + /**
          + * Create a block.
          + * @param limit limit of the block.
          + * @return a new block.
          + */
          + abstract DataBlock create(int limit) throws IOException;
          +
          + /**
          + * Implement any close/cleanup operation.
          + * Base class is a no-op
          + * @throws IOException -ideally, it shouldn't.
          + */
          + @Override
          + public void close() throws IOException

          { + }

          + }
          +
          + /**
          + * This represents a block being uploaded.
          + */
          + static abstract class DataBlock implements Closeable {
          +
          + private volatile DestState state = Writing;
          +
          + /**
          + * Enter
          + * @param current
          + * @param next
          + * @throws IllegalStateException
          + */
          + protected synchronized final void enterState(DestState current,
          + DestState next)
          + throws IllegalStateException {
          + verifyState(current);
          + LOG.debug("{}: entering state {}", this, next);
          + state = next;
          + }
          +
          + /**
          + * Verify that the block is in the declared state.
          + * @param expected expected state.
          + * @throws IllegalStateException if the DataBlock is in the wrong state
          + */
          + protected final void verifyState(DestState expected) throws IllegalStateException {
          + if (expected != null && state != expected)

          { + throw new IllegalStateException("Expected stream state " + expected + + " -but actual state is " + state + " in " + this); + }

          + }
          +
          + /**
          + * Current state.
          + * @return the current state.
          + */
          + final DestState getState()

          { + return state; + }

          +
          + /**
          + * Return the current data size.
          + * @return the size of the data
          + */
          + abstract int dataSize();
          +
          + /**
          + * Predicate to verify that the block has the capacity to write
          + * the given set of bytes.
          + * @param bytes number of bytes desired to be written.
          + * @return true if there is enough space.
          + */
          + abstract boolean hasCapacity(long bytes);
          +
          + /**
          + * Predicate to check if there is data in the block.
          + * @return true if there is
          + */
          + boolean hasData()

          { + return dataSize() > 0; + }

          +
          + /**
          + * The remaining capacity in the block before it is full.
          + * @return the number of bytes remaining.
          + */
          + abstract int remainingCapacity();
          +
          + /**
          + * Write a series of bytes from the buffer, from the offset.
          + * Returns the number of bytes written.
          + * Only valid in the state

          {@code Writing}.
          + * Base class verifies the state but does no writing.
          + * @param buffer buffer
          + * @param offset offset
          + * @param length length of write
          + * @return number of bytes written
          + * @throws IOException trouble
          + */
          + int write(byte[] buffer, int offset, int length) throws IOException { + verifyState(Writing); + Preconditions.checkArgument(buffer != null, "Null buffer"); + Preconditions.checkArgument(length >= 0, "length is negative"); + Preconditions.checkArgument(offset >= 0, "offset is negative"); + Preconditions.checkArgument( + !(buffer.length - offset < length), + "buffer shorter than amount of data to write"); + return 0; + }
          +
          + /**
          + * Flush the output.
          + * Only valid in the state {@code Writing}

          .
          + * In the base class, this is a no-op
          + * @throws IOException any IO problem.
          + */
          + void flush() throws IOException

          { + verifyState(Writing); + }

          +
          + /**
          + * Switch to the upload state and return a stream for uploading.
          + * Base class calls

          {@link #enterState(DestState, DestState)}

          to
          + * manage the state machine.
          + * @return the stream
          + * @throws IOException trouble
          + */
          + InputStream startUpload() throws IOException

          { + LOG.debug("Start datablock upload"); + enterState(Writing, Upload); + return null; + }

          +
          + /**
          + * Enter the closed state.
          + * @return true if the class was in any other state, implying that
          + * the subclass should do its close operations
          + */
          + protected synchronized boolean enterClosedState() {
          + if (!state.equals(Closed)) {
          + try

          { + enterState(null, Closed); + }

          catch (IllegalStateException ignored)

          { + + }

          + return true;
          + } else

          { + return false; + }

          + }
          +
          + @Override
          + public void close() throws IOException {
          + if (enterClosedState()) {
          + LOG.debug("Closed {}", this);
          + innerClose();
          + }
          + }
          +
          + /**
          + * Inner close logic for subclasses to implement.
          + */
          + protected void innerClose() throws IOException

          { + + }

          +
          + enum DestState

          {Writing, Upload, Closed}

          + }
          +
          + // ====================================================================
          +
          + /**
          + * Use byte arrays on the heap for storage.
          + */
          + static class ArrayBlockFactory extends BlockFactory {
          +
          + ArrayBlockFactory(S3AFileSystem owner)

          { + super(owner); + }

          +
          + @Override
          + DataBlock create(int limit) throws IOException

          { + return new ByteArrayBlock(limit); + }

          +
          + }
          +
          + /**
          + * Stream to memory via a

          {@code ByteArrayOutputStream}

          .
          + *
          + * This was taken from

          {@code S3AFastOutputStream}

          and has the
          + * same problem which surfaced there: it consumes heap space
          + * proportional to the mismatch between writes to the stream and
          + * the JVM-wide upload bandwidth to the S3 endpoint.
          + */
          +
          + static class ByteArrayBlock extends DataBlock {
          + private ByteArrayOutputStream buffer;
          + private final int limit;
          + // cache data size so that it is consistent after the buffer is reset.
          + private Integer dataSize;
          +
          + ByteArrayBlock(int limit)

          { + this.limit = limit; + buffer = new ByteArrayOutputStream(); + }

          +
          + /**
          + * Get the amount of data; if there is no buffer then the size is 0.
          + * @return the amount of data available to upload.
          + */
          + @Override
          + int dataSize()

          { + return dataSize != null ? dataSize : buffer.size(); + }

          +
          + @Override
          + InputStream startUpload() throws IOException {
          + super.startUpload();
          + dataSize = buffer.size();
          + ByteArrayInputStream bufferData = new ByteArrayInputStream(
          + buffer.toByteArray());
          + buffer.reset();
          — End diff –

          I was thinking you could remove the `buffer.reset()`, because the next line is dropping the reference to `buffer` anyway.

          Show
          githubbot ASF GitHub Bot added a comment - Github user cnauroth commented on a diff in the pull request: https://github.com/apache/hadoop/pull/130#discussion_r82087028 — Diff: hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ADataBlocks.java — @@ -0,0 +1,816 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.s3a; + +import java.io.BufferedOutputStream; +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.Closeable; +import java.io.EOFException; +import java.io.File; +import java.io.FileInputStream; +import java.io.FileNotFoundException; +import java.io.FileOutputStream; +import java.io.FilterInputStream; +import java.io.IOException; +import java.io.InputStream; +import java.nio.BufferUnderflowException; +import java.nio.ByteBuffer; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; + +import com.google.common.base.Preconditions; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.fs.FSExceptionMessages; +import org.apache.hadoop.util.DirectBufferPool; + +import static org.apache.hadoop.fs.s3a.S3ADataBlocks.DataBlock.DestState.*; + +/** + * Set of classes to support output streaming into blocks which are then + * uploaded as partitions. + */ +final class S3ADataBlocks { + + private static final Logger LOG = LoggerFactory.getLogger(S3ADataBlocks.class); + + private S3ADataBlocks() { + } + + /** + * Validate args to a write command. These are the same validation checks + * expected for any implementation of {@code OutputStream.write()} . + * @param b byte array containing data + * @param off offset in array where to start + * @param len number of bytes to be written + * @throws NullPointerException for a null buffer + * @throws IndexOutOfBoundsException if indices are out of range + */ + static void validateWriteArgs(byte[] b, int off, int len) + throws IOException { + Preconditions.checkNotNull(b); + if ((off < 0) || (off > b.length) || (len < 0) || + ((off + len) > b.length) || ((off + len) < 0)) { + throw new IndexOutOfBoundsException( + "write (b[" + b.length + "], " + off + ", " + len + ')'); + } + } + + /** + * Create a factory. + * @param owner factory owner + * @param name factory name -the option from {@link Constants} . + * @return the factory, ready to be initialized. + * @throws IllegalArgumentException if the name is unknown. + */ + static BlockFactory createFactory(S3AFileSystem owner, + String name) { + switch (name) { + case Constants.BLOCK_OUTPUT_BUFFER_ARRAY: + return new ArrayBlockFactory(owner); + case Constants.BLOCK_OUTPUT_BUFFER_DISK: + return new DiskBlockFactory(owner); + case Constants.BLOCK_OUTPUT_BYTEBUFFER: + return new ByteBufferBlockFactory(owner); + default: + throw new IllegalArgumentException("Unsupported block buffer" + + " \"" + name + '"'); + } + } + + /** + * Base class for block factories. + */ + static abstract class BlockFactory implements Closeable { + + /** + * Owner. + */ + protected final S3AFileSystem owner; + + protected BlockFactory(S3AFileSystem owner) { + this.owner = owner; + } + + /** + * Create a block. + * @param limit limit of the block. + * @return a new block. + */ + abstract DataBlock create(int limit) throws IOException; + + /** + * Implement any close/cleanup operation. + * Base class is a no-op + * @throws IOException -ideally, it shouldn't. + */ + @Override + public void close() throws IOException { + } + } + + /** + * This represents a block being uploaded. + */ + static abstract class DataBlock implements Closeable { + + private volatile DestState state = Writing; + + /** + * Enter + * @param current + * @param next + * @throws IllegalStateException + */ + protected synchronized final void enterState(DestState current, + DestState next) + throws IllegalStateException { + verifyState(current); + LOG.debug("{}: entering state {}", this, next); + state = next; + } + + /** + * Verify that the block is in the declared state. + * @param expected expected state. + * @throws IllegalStateException if the DataBlock is in the wrong state + */ + protected final void verifyState(DestState expected) throws IllegalStateException { + if (expected != null && state != expected) { + throw new IllegalStateException("Expected stream state " + expected + + " -but actual state is " + state + " in " + this); + } + } + + /** + * Current state. + * @return the current state. + */ + final DestState getState() { + return state; + } + + /** + * Return the current data size. + * @return the size of the data + */ + abstract int dataSize(); + + /** + * Predicate to verify that the block has the capacity to write + * the given set of bytes. + * @param bytes number of bytes desired to be written. + * @return true if there is enough space. + */ + abstract boolean hasCapacity(long bytes); + + /** + * Predicate to check if there is data in the block. + * @return true if there is + */ + boolean hasData() { + return dataSize() > 0; + } + + /** + * The remaining capacity in the block before it is full. + * @return the number of bytes remaining. + */ + abstract int remainingCapacity(); + + /** + * Write a series of bytes from the buffer, from the offset. + * Returns the number of bytes written. + * Only valid in the state {@code Writing}. + * Base class verifies the state but does no writing. + * @param buffer buffer + * @param offset offset + * @param length length of write + * @return number of bytes written + * @throws IOException trouble + */ + int write(byte[] buffer, int offset, int length) throws IOException { + verifyState(Writing); + Preconditions.checkArgument(buffer != null, "Null buffer"); + Preconditions.checkArgument(length >= 0, "length is negative"); + Preconditions.checkArgument(offset >= 0, "offset is negative"); + Preconditions.checkArgument( + !(buffer.length - offset < length), + "buffer shorter than amount of data to write"); + return 0; + } + + /** + * Flush the output. + * Only valid in the state {@code Writing} . + * In the base class, this is a no-op + * @throws IOException any IO problem. + */ + void flush() throws IOException { + verifyState(Writing); + } + + /** + * Switch to the upload state and return a stream for uploading. + * Base class calls {@link #enterState(DestState, DestState)} to + * manage the state machine. + * @return the stream + * @throws IOException trouble + */ + InputStream startUpload() throws IOException { + LOG.debug("Start datablock upload"); + enterState(Writing, Upload); + return null; + } + + /** + * Enter the closed state. + * @return true if the class was in any other state, implying that + * the subclass should do its close operations + */ + protected synchronized boolean enterClosedState() { + if (!state.equals(Closed)) { + try { + enterState(null, Closed); + } catch (IllegalStateException ignored) { + + } + return true; + } else { + return false; + } + } + + @Override + public void close() throws IOException { + if (enterClosedState()) { + LOG.debug("Closed {}", this); + innerClose(); + } + } + + /** + * Inner close logic for subclasses to implement. + */ + protected void innerClose() throws IOException { + + } + + enum DestState {Writing, Upload, Closed} + } + + // ==================================================================== + + /** + * Use byte arrays on the heap for storage. + */ + static class ArrayBlockFactory extends BlockFactory { + + ArrayBlockFactory(S3AFileSystem owner) { + super(owner); + } + + @Override + DataBlock create(int limit) throws IOException { + return new ByteArrayBlock(limit); + } + + } + + /** + * Stream to memory via a {@code ByteArrayOutputStream} . + * + * This was taken from {@code S3AFastOutputStream} and has the + * same problem which surfaced there: it consumes heap space + * proportional to the mismatch between writes to the stream and + * the JVM-wide upload bandwidth to the S3 endpoint. + */ + + static class ByteArrayBlock extends DataBlock { + private ByteArrayOutputStream buffer; + private final int limit; + // cache data size so that it is consistent after the buffer is reset. + private Integer dataSize; + + ByteArrayBlock(int limit) { + this.limit = limit; + buffer = new ByteArrayOutputStream(); + } + + /** + * Get the amount of data; if there is no buffer then the size is 0. + * @return the amount of data available to upload. + */ + @Override + int dataSize() { + return dataSize != null ? dataSize : buffer.size(); + } + + @Override + InputStream startUpload() throws IOException { + super.startUpload(); + dataSize = buffer.size(); + ByteArrayInputStream bufferData = new ByteArrayInputStream( + buffer.toByteArray()); + buffer.reset(); — End diff – I was thinking you could remove the `buffer.reset()`, because the next line is dropping the reference to `buffer` anyway.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user cnauroth commented on a diff in the pull request:

          https://github.com/apache/hadoop/pull/130#discussion_r82091762

          — Diff: hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java —
          @@ -0,0 +1,699 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +
          +package org.apache.hadoop.fs.s3a;
          +
          +import java.io.IOException;
          +import java.io.InputStream;
          +import java.io.OutputStream;
          +import java.util.ArrayList;
          +import java.util.List;
          +import java.util.concurrent.Callable;
          +import java.util.concurrent.ExecutionException;
          +import java.util.concurrent.ExecutorService;
          +import java.util.concurrent.TimeUnit;
          +import java.util.concurrent.atomic.AtomicBoolean;
          +
          +import com.amazonaws.AmazonClientException;
          +import com.amazonaws.event.ProgressEvent;
          +import com.amazonaws.event.ProgressEventType;
          +import com.amazonaws.event.ProgressListener;
          +import com.amazonaws.services.s3.model.CompleteMultipartUploadResult;
          +import com.amazonaws.services.s3.model.PartETag;
          +import com.amazonaws.services.s3.model.PutObjectRequest;
          +import com.amazonaws.services.s3.model.PutObjectResult;
          +import com.amazonaws.services.s3.model.UploadPartRequest;
          +import com.google.common.base.Preconditions;
          +import com.google.common.util.concurrent.Futures;
          +import com.google.common.util.concurrent.ListenableFuture;
          +import com.google.common.util.concurrent.ListeningExecutorService;
          +import com.google.common.util.concurrent.MoreExecutors;
          +import org.slf4j.Logger;
          +import org.slf4j.LoggerFactory;
          +
          +import org.apache.hadoop.classification.InterfaceAudience;
          +import org.apache.hadoop.classification.InterfaceStability;
          +import org.apache.hadoop.io.IOUtils;
          +import org.apache.hadoop.io.retry.RetryPolicies;
          +import org.apache.hadoop.io.retry.RetryPolicy;
          +import org.apache.hadoop.util.Progressable;
          +
          +import static org.apache.hadoop.fs.s3a.S3AUtils.*;
          +import static org.apache.hadoop.fs.s3a.Statistic.*;
          +
          +/**
          + * Upload files/parts directly via different buffering mechanisms:
          + * including memory and disk.
          + *
          + * If the stream is closed and no update has started, then the upload
          + * is instead done as a single PUT operation.
          + *
          + * Unstable: statistics and error handling might evolve.
          + */
          +@InterfaceAudience.Private
          +@InterfaceStability.Unstable
          +class S3ABlockOutputStream extends OutputStream {
          +
          + private static final Logger LOG =
          + LoggerFactory.getLogger(S3ABlockOutputStream.class);
          +
          + /** Owner FileSystem. */
          + private final S3AFileSystem fs;
          +
          + /** Object being uploaded. */
          + private final String key;
          +
          + /** Size of all blocks. */
          + private final int blockSize;
          +
          + /** Callback for progress. */
          + private final ProgressListener progressListener;
          + private final ListeningExecutorService executorService;
          +
          + /**
          + * Retry policy for multipart commits; not all AWS SDK versions retry that.
          + */
          + private final RetryPolicy retryPolicy =
          + RetryPolicies.retryUpToMaximumCountWithProportionalSleep(
          + 5,
          + 2000,
          + TimeUnit.MILLISECONDS);
          + /**
          + * Factory for blocks.
          + */
          + private final S3ADataBlocks.BlockFactory blockFactory;
          +
          + /** Preallocated byte buffer for writing single characters. */
          + private final byte[] singleCharWrite = new byte[1];
          +
          + /** Multipart upload details; null means none started. */
          + private MultiPartUpload multiPartUpload;
          +
          + /** Closed flag. */
          + private final AtomicBoolean closed = new AtomicBoolean(false);
          +
          + /** Current data block. Null means none currently active */
          + private S3ADataBlocks.DataBlock activeBlock;
          +
          + /** Count of blocks uploaded. */
          + private long blockCount = 0;
          +
          + /** Statistics to build up. */
          + private final S3AInstrumentation.OutputStreamStatistics statistics;
          +
          + /**
          + * Write operation helper; encapsulation of the filesystem operations.
          + */
          + private final S3AFileSystem.WriteOperationHelper writeOperationHelper;
          +
          + /**
          + * An S3A output stream which uploads partitions in a separate pool of
          + * threads; different

          {@link S3ADataBlocks.BlockFactory}

          + * instances can control where data is buffered.
          + *
          + * @param fs S3AFilesystem
          + * @param key S3 object to work on.
          + * @param executorService the executor service to use to schedule work
          + * @param progress report progress in order to prevent timeouts. If
          + * this class implements

          {@code ProgressListener}

          then it will be
          + * directly wired up to the AWS client, so receive detailed progress
          + * information.
          + * @param blockSize size of a single block.
          + * @param blockFactory factory for creating stream destinations
          + * @param statistics stats for this stream
          + * @param writeOperationHelper state of the write operation.
          + * @throws IOException on any problem
          + */
          + S3ABlockOutputStream(S3AFileSystem fs,
          + String key,
          + ExecutorService executorService,
          + Progressable progress,
          + long blockSize,
          + S3ADataBlocks.BlockFactory blockFactory,
          + S3AInstrumentation.OutputStreamStatistics statistics,
          + S3AFileSystem.WriteOperationHelper writeOperationHelper)
          + throws IOException {
          + this.fs = fs;
          + this.key = key;
          + this.blockFactory = blockFactory;
          + this.blockSize = (int) blockSize;
          + this.statistics = statistics;
          + this.writeOperationHelper = writeOperationHelper;
          + Preconditions.checkArgument(blockSize >= Constants.MULTIPART_MIN_SIZE,
          + "Block size is too small: %d", blockSize);
          + this.executorService = MoreExecutors.listeningDecorator(executorService);
          + this.multiPartUpload = null;
          + this.progressListener = (progress instanceof ProgressListener) ?
          + (ProgressListener) progress
          + : new ProgressableListener(progress);
          + LOG.debug("Initialized S3ABlockOutputStream for {}" +
          — End diff –

          I think `activeBlock` is always `null` when this log statement executes.

          Show
          githubbot ASF GitHub Bot added a comment - Github user cnauroth commented on a diff in the pull request: https://github.com/apache/hadoop/pull/130#discussion_r82091762 — Diff: hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java — @@ -0,0 +1,699 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.s3a; + +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.Callable; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; + +import com.amazonaws.AmazonClientException; +import com.amazonaws.event.ProgressEvent; +import com.amazonaws.event.ProgressEventType; +import com.amazonaws.event.ProgressListener; +import com.amazonaws.services.s3.model.CompleteMultipartUploadResult; +import com.amazonaws.services.s3.model.PartETag; +import com.amazonaws.services.s3.model.PutObjectRequest; +import com.amazonaws.services.s3.model.PutObjectResult; +import com.amazonaws.services.s3.model.UploadPartRequest; +import com.google.common.base.Preconditions; +import com.google.common.util.concurrent.Futures; +import com.google.common.util.concurrent.ListenableFuture; +import com.google.common.util.concurrent.ListeningExecutorService; +import com.google.common.util.concurrent.MoreExecutors; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.io.IOUtils; +import org.apache.hadoop.io.retry.RetryPolicies; +import org.apache.hadoop.io.retry.RetryPolicy; +import org.apache.hadoop.util.Progressable; + +import static org.apache.hadoop.fs.s3a.S3AUtils.*; +import static org.apache.hadoop.fs.s3a.Statistic.*; + +/** + * Upload files/parts directly via different buffering mechanisms: + * including memory and disk. + * + * If the stream is closed and no update has started, then the upload + * is instead done as a single PUT operation. + * + * Unstable: statistics and error handling might evolve. + */ +@InterfaceAudience.Private +@InterfaceStability.Unstable +class S3ABlockOutputStream extends OutputStream { + + private static final Logger LOG = + LoggerFactory.getLogger(S3ABlockOutputStream.class); + + /** Owner FileSystem. */ + private final S3AFileSystem fs; + + /** Object being uploaded. */ + private final String key; + + /** Size of all blocks. */ + private final int blockSize; + + /** Callback for progress. */ + private final ProgressListener progressListener; + private final ListeningExecutorService executorService; + + /** + * Retry policy for multipart commits; not all AWS SDK versions retry that. + */ + private final RetryPolicy retryPolicy = + RetryPolicies.retryUpToMaximumCountWithProportionalSleep( + 5, + 2000, + TimeUnit.MILLISECONDS); + /** + * Factory for blocks. + */ + private final S3ADataBlocks.BlockFactory blockFactory; + + /** Preallocated byte buffer for writing single characters. */ + private final byte[] singleCharWrite = new byte [1] ; + + /** Multipart upload details; null means none started. */ + private MultiPartUpload multiPartUpload; + + /** Closed flag. */ + private final AtomicBoolean closed = new AtomicBoolean(false); + + /** Current data block. Null means none currently active */ + private S3ADataBlocks.DataBlock activeBlock; + + /** Count of blocks uploaded. */ + private long blockCount = 0; + + /** Statistics to build up. */ + private final S3AInstrumentation.OutputStreamStatistics statistics; + + /** + * Write operation helper; encapsulation of the filesystem operations. + */ + private final S3AFileSystem.WriteOperationHelper writeOperationHelper; + + /** + * An S3A output stream which uploads partitions in a separate pool of + * threads; different {@link S3ADataBlocks.BlockFactory} + * instances can control where data is buffered. + * + * @param fs S3AFilesystem + * @param key S3 object to work on. + * @param executorService the executor service to use to schedule work + * @param progress report progress in order to prevent timeouts. If + * this class implements {@code ProgressListener} then it will be + * directly wired up to the AWS client, so receive detailed progress + * information. + * @param blockSize size of a single block. + * @param blockFactory factory for creating stream destinations + * @param statistics stats for this stream + * @param writeOperationHelper state of the write operation. + * @throws IOException on any problem + */ + S3ABlockOutputStream(S3AFileSystem fs, + String key, + ExecutorService executorService, + Progressable progress, + long blockSize, + S3ADataBlocks.BlockFactory blockFactory, + S3AInstrumentation.OutputStreamStatistics statistics, + S3AFileSystem.WriteOperationHelper writeOperationHelper) + throws IOException { + this.fs = fs; + this.key = key; + this.blockFactory = blockFactory; + this.blockSize = (int) blockSize; + this.statistics = statistics; + this.writeOperationHelper = writeOperationHelper; + Preconditions.checkArgument(blockSize >= Constants.MULTIPART_MIN_SIZE, + "Block size is too small: %d", blockSize); + this.executorService = MoreExecutors.listeningDecorator(executorService); + this.multiPartUpload = null; + this.progressListener = (progress instanceof ProgressListener) ? + (ProgressListener) progress + : new ProgressableListener(progress); + LOG.debug("Initialized S3ABlockOutputStream for {}" + — End diff – I think `activeBlock` is always `null` when this log statement executes.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user cnauroth commented on a diff in the pull request:

          https://github.com/apache/hadoop/pull/130#discussion_r82093069

          — Diff: hadoop-common-project/hadoop-common/src/main/resources/core-default.xml —
          @@ -1093,12 +1101,48 @@
          </property>

          <property>

          • <name>fs.s3a.fast.upload</name>
            + <name>fs.s3a.block.output</name>
              • End diff –

          Is this revision missing the changes to restore/un-deprecate `fs.s3a.fast.upload`?

          Show
          githubbot ASF GitHub Bot added a comment - Github user cnauroth commented on a diff in the pull request: https://github.com/apache/hadoop/pull/130#discussion_r82093069 — Diff: hadoop-common-project/hadoop-common/src/main/resources/core-default.xml — @@ -1093,12 +1101,48 @@ </property> <property> <name>fs.s3a.fast.upload</name> + <name>fs.s3a.block.output</name> End diff – Is this revision missing the changes to restore/un-deprecate `fs.s3a.fast.upload`?
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user cnauroth commented on a diff in the pull request:

          https://github.com/apache/hadoop/pull/130#discussion_r82086784

          — Diff: hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ADataBlocks.java —
          @@ -0,0 +1,816 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +
          +package org.apache.hadoop.fs.s3a;
          +
          +import java.io.BufferedOutputStream;
          +import java.io.ByteArrayInputStream;
          +import java.io.ByteArrayOutputStream;
          +import java.io.Closeable;
          +import java.io.EOFException;
          +import java.io.File;
          +import java.io.FileInputStream;
          +import java.io.FileNotFoundException;
          +import java.io.FileOutputStream;
          +import java.io.FilterInputStream;
          +import java.io.IOException;
          +import java.io.InputStream;
          +import java.nio.BufferUnderflowException;
          +import java.nio.ByteBuffer;
          +import java.util.concurrent.atomic.AtomicBoolean;
          +import java.util.concurrent.atomic.AtomicInteger;
          +
          +import com.google.common.base.Preconditions;
          +import org.slf4j.Logger;
          +import org.slf4j.LoggerFactory;
          +
          +import org.apache.hadoop.fs.FSExceptionMessages;
          +import org.apache.hadoop.util.DirectBufferPool;
          +
          +import static org.apache.hadoop.fs.s3a.S3ADataBlocks.DataBlock.DestState.*;
          +
          +/**
          + * Set of classes to support output streaming into blocks which are then
          + * uploaded as partitions.
          + */
          +final class S3ADataBlocks {
          +
          + private static final Logger LOG = LoggerFactory.getLogger(S3ADataBlocks.class);
          +
          + private S3ADataBlocks()

          { + }

          +
          + /**
          + * Validate args to a write command. These are the same validation checks
          + * expected for any implementation of

          {@code OutputStream.write()}

          .
          + * @param b byte array containing data
          + * @param off offset in array where to start
          + * @param len number of bytes to be written
          + * @throws NullPointerException for a null buffer
          + * @throws IndexOutOfBoundsException if indices are out of range
          + */
          + static void validateWriteArgs(byte[] b, int off, int len)
          + throws IOException {
          + Preconditions.checkNotNull(b);
          + if ((off < 0) || (off > b.length) || (len < 0) ||
          + ((off + len) > b.length) || ((off + len) < 0))

          { + throw new IndexOutOfBoundsException( + "write (b[" + b.length + "], " + off + ", " + len + ')'); + }

          + }
          +
          + /**
          + * Create a factory.
          + * @param owner factory owner
          + * @param name factory name -the option from

          {@link Constants}

          .
          + * @return the factory, ready to be initialized.
          + * @throws IllegalArgumentException if the name is unknown.
          + */
          + static BlockFactory createFactory(S3AFileSystem owner,
          + String name) {
          + switch (name)

          { + case Constants.BLOCK_OUTPUT_BUFFER_ARRAY: + return new ArrayBlockFactory(owner); + case Constants.BLOCK_OUTPUT_BUFFER_DISK: + return new DiskBlockFactory(owner); + case Constants.BLOCK_OUTPUT_BYTEBUFFER: + return new ByteBufferBlockFactory(owner); + default: + throw new IllegalArgumentException("Unsupported block buffer" + + " \"" + name + '"'); + }

          + }
          +
          + /**
          + * Base class for block factories.
          + */
          + static abstract class BlockFactory implements Closeable {
          +
          + /**
          + * Owner.
          + */
          + protected final S3AFileSystem owner;
          +
          + protected BlockFactory(S3AFileSystem owner)

          { + this.owner = owner; + }

          +
          + /**
          + * Create a block.
          + * @param limit limit of the block.
          + * @return a new block.
          + */
          + abstract DataBlock create(int limit) throws IOException;
          +
          + /**
          + * Implement any close/cleanup operation.
          + * Base class is a no-op
          + * @throws IOException -ideally, it shouldn't.
          + */
          + @Override
          + public void close() throws IOException

          { + }

          + }
          +
          + /**
          + * This represents a block being uploaded.
          + */
          + static abstract class DataBlock implements Closeable {
          +
          + private volatile DestState state = Writing;
          +
          + /**
          + * Enter
          + * @param current
          + * @param next
          + * @throws IllegalStateException
          + */
          + protected synchronized final void enterState(DestState current,
          + DestState next)
          + throws IllegalStateException {
          + verifyState(current);
          + LOG.debug("{}: entering state {}", this, next);
          + state = next;
          + }
          +
          + /**
          + * Verify that the block is in the declared state.
          + * @param expected expected state.
          + * @throws IllegalStateException if the DataBlock is in the wrong state
          + */
          + protected final void verifyState(DestState expected) throws IllegalStateException {
          + if (expected != null && state != expected)

          { + throw new IllegalStateException("Expected stream state " + expected + + " -but actual state is " + state + " in " + this); + }

          + }
          +
          + /**
          + * Current state.
          + * @return the current state.
          + */
          + final DestState getState()

          { + return state; + }

          +
          + /**
          + * Return the current data size.
          + * @return the size of the data
          + */
          + abstract int dataSize();
          +
          + /**
          + * Predicate to verify that the block has the capacity to write
          + * the given set of bytes.
          + * @param bytes number of bytes desired to be written.
          + * @return true if there is enough space.
          + */
          + abstract boolean hasCapacity(long bytes);
          +
          + /**
          + * Predicate to check if there is data in the block.
          + * @return true if there is
          + */
          + boolean hasData()

          { + return dataSize() > 0; + }

          +
          + /**
          + * The remaining capacity in the block before it is full.
          + * @return the number of bytes remaining.
          + */
          + abstract int remainingCapacity();
          +
          + /**
          + * Write a series of bytes from the buffer, from the offset.
          + * Returns the number of bytes written.
          + * Only valid in the state

          {@code Writing}.
          + * Base class verifies the state but does no writing.
          + * @param buffer buffer
          + * @param offset offset
          + * @param length length of write
          + * @return number of bytes written
          + * @throws IOException trouble
          + */
          + int write(byte[] buffer, int offset, int length) throws IOException { + verifyState(Writing); + Preconditions.checkArgument(buffer != null, "Null buffer"); + Preconditions.checkArgument(length >= 0, "length is negative"); + Preconditions.checkArgument(offset >= 0, "offset is negative"); + Preconditions.checkArgument( + !(buffer.length - offset < length), + "buffer shorter than amount of data to write"); + return 0; + }
          +
          + /**
          + * Flush the output.
          + * Only valid in the state {@code Writing}

          .
          + * In the base class, this is a no-op
          + * @throws IOException any IO problem.
          + */
          + void flush() throws IOException

          { + verifyState(Writing); + }

          +
          + /**
          + * Switch to the upload state and return a stream for uploading.
          + * Base class calls

          {@link #enterState(DestState, DestState)}

          to
          + * manage the state machine.
          + * @return the stream
          + * @throws IOException trouble
          + */
          + InputStream startUpload() throws IOException

          { + LOG.debug("Start datablock upload"); + enterState(Writing, Upload); + return null; + }

          +
          + /**
          + * Enter the closed state.
          + * @return true if the class was in any other state, implying that
          + * the subclass should do its close operations
          + */
          + protected synchronized boolean enterClosedState() {
          + if (!state.equals(Closed)) {
          + try

          { + enterState(null, Closed); + }

          catch (IllegalStateException ignored)

          { + + }

          + return true;
          + } else

          { + return false; + }

          + }
          +
          + @Override
          + public void close() throws IOException {
          + if (enterClosedState()) {
          + LOG.debug("Closed {}", this);
          + innerClose();
          + }
          + }
          +
          + /**
          + * Inner close logic for subclasses to implement.
          + */
          + protected void innerClose() throws IOException

          { + + }

          +
          + enum DestState

          {Writing, Upload, Closed}

          + }
          +
          + // ====================================================================
          +
          + /**
          + * Use byte arrays on the heap for storage.
          + */
          + static class ArrayBlockFactory extends BlockFactory {
          +
          + ArrayBlockFactory(S3AFileSystem owner)

          { + super(owner); + }
          +
          + @Override
          + DataBlock create(int limit) throws IOException { + return new ByteArrayBlock(limit); + }
          +
          + }
          +
          + /**
          + * Stream to memory via a {@code ByteArrayOutputStream}.
          + *
          + * This was taken from {@code S3AFastOutputStream} and has the
          + * same problem which surfaced there: it consumes heap space
          + * proportional to the mismatch between writes to the stream and
          + * the JVM-wide upload bandwidth to the S3 endpoint.
          + */
          +
          + static class ByteArrayBlock extends DataBlock {
          + private ByteArrayOutputStream buffer;
          + private final int limit;
          + // cache data size so that it is consistent after the buffer is reset.
          + private Integer dataSize;
          +
          + ByteArrayBlock(int limit) { + this.limit = limit; + buffer = new ByteArrayOutputStream(); + }
          +
          + /**
          + * Get the amount of data; if there is no buffer then the size is 0.
          + * @return the amount of data available to upload.
          + */
          + @Override
          + int dataSize() { + return dataSize != null ? dataSize : buffer.size(); + }
          +
          + @Override
          + InputStream startUpload() throws IOException { + super.startUpload(); + dataSize = buffer.size(); + ByteArrayInputStream bufferData = new ByteArrayInputStream( + buffer.toByteArray()); + buffer.reset(); + buffer = null; + return bufferData; + }
          +
          + @Override
          + boolean hasCapacity(long bytes) { + return dataSize() + bytes <= limit; + }
          +
          + @Override
          + int remainingCapacity() { + return limit - dataSize(); + }
          +
          + @Override
          + int write(byte[] b, int offset, int len) throws IOException { + super.write(b, offset, len); + int written = Math.min(remainingCapacity(), len); + buffer.write(b, offset, written); + return written; + }
          +
          + @Override
          + protected void innerClose() { + buffer = null; + }
          +
          + @Override
          + public String toString() {
          + return "ByteArrayBlock{" + + "state=" + getState() + + ", limit=" + limit + + ", dataSize=" + dataSize + + '}';
          + }
          + }
          +
          + // ====================================================================
          +
          + /**
          + * Stream via Direct ByteBuffers; these are allocated off heap
          + * via {@link DirectBufferPool}.
          + * This is actually the most complex of all the block factories,
          + * due to the need to explicitly recycle buffers; in comparison, the
          + * {@link DiskBlock} buffer delegates the work of deleting files to
          + * the {@link DiskBlock.FileDeletingInputStream}. Here the
          + * input stream {@link ByteBufferInputStream} has a similar task, along
          + * with the foundational work of streaming data from a byte array.
          + */
          +
          + static class ByteBufferBlockFactory extends BlockFactory {
          +
          + private final DirectBufferPool bufferPool = new DirectBufferPool();
          + private final AtomicInteger buffersOutstanding = new AtomicInteger(0);
          +
          + ByteBufferBlockFactory(S3AFileSystem owner) { + super(owner); + }

          +
          + @Override
          + ByteBufferBlock create(int limit) throws IOException

          { + return new ByteBufferBlock(limit); + }

          +
          + private ByteBuffer requestBuffer(int limit) {
          + LOG.debug("Requesting buffer of size {}", limit);
          + buffersOutstanding.incrementAndGet();
          + return bufferPool.getBuffer(limit);
          + }
          +
          + private void releaseBuffer(ByteBuffer buffer)

          { + LOG.debug("Releasing buffer"); + bufferPool.returnBuffer(buffer); + buffersOutstanding.decrementAndGet(); + }

          +
          + /**
          + * Get count of outstanding buffers.
          + * @return the current buffer count
          + */
          + public int getOutstandingBufferCount()

          { + return buffersOutstanding.get(); + }

          +
          + @Override
          + public String toString() {
          + return "ByteBufferBlockFactory

          {" + + "buffersOutstanding=" + buffersOutstanding + + '}

          ';
          + }
          +
          + /**
          + * A DataBlock which requests a buffer from pool on creation; returns
          + * it when the output stream is closed.
          + */
          + class ByteBufferBlock extends DataBlock {
          + private ByteBuffer buffer;
          + private final int bufferSize;
          + // cache data size so that it is consistent after the buffer is reset.
          + private Integer dataSize;
          +
          + /**
          + * Instantiate. This will request a ByteBuffer of the desired size.
          + * @param bufferSize buffer size
          + */
          + ByteBufferBlock(int bufferSize)

          { + this.bufferSize = bufferSize; + buffer = requestBuffer(bufferSize); + }

          +
          + /**
          + * Get the amount of data; if there is no buffer then the size is 0.
          + * @return the amount of data available to upload.
          + */
          + @Override
          + int dataSize()

          { + return dataSize != null ? dataSize : bufferCapacityUsed(); + }

          +
          + @Override
          + ByteBufferInputStream startUpload() throws IOException

          { + super.startUpload(); + dataSize = bufferCapacityUsed(); + // set the buffer up from reading from the beginning + buffer.limit(buffer.position()); + buffer.position(0); + return new ByteBufferInputStream(dataSize, buffer); + }

          +
          + @Override
          + public boolean hasCapacity(long bytes)

          { + return bytes <= remainingCapacity(); + }

          +
          + @Override
          + public int remainingCapacity()

          { + return buffer != null ? buffer.remaining() : 0; + }

          +
          + private int bufferCapacityUsed()

          { + return buffer.capacity() - buffer.remaining(); + }

          +
          + @Override
          + int write(byte[] b, int offset, int len) throws IOException

          { + super.write(b, offset, len); + int written = Math.min(remainingCapacity(), len); + buffer.put(b, offset, written); + return written; + }

          +
          + @Override
          + protected void innerClose()

          { + buffer = null; + }

          +
          + @Override
          + public String toString() {
          + return "ByteBufferBlock

          {" + + "state=" + getState() + + ", dataSize=" + dataSize() + + ", limit=" + bufferSize + + ", remainingCapacity=" + remainingCapacity() + + '}

          ';
          + }
          +
          + }
          +
          + /**
          + * Provide an input stream from a byte buffer; supporting
          + *

          {@link #mark(int)}

          , which is required to enable replay of failed
          + * PUT attempts.
          + * This input stream returns the buffer to the pool afterwards.
          + */
          + class ByteBufferInputStream extends InputStream {
          +
          + private final int size;
          + private ByteBuffer byteBuffer;
          +
          + ByteBufferInputStream(int size, ByteBuffer byteBuffer) {
          + LOG.debug("Creating ByteBufferInputStream of size {}", size);
          + this.size = size;
          + this.byteBuffer = byteBuffer;
          + }
          +
          + /**
          + * Return the buffer to the pool after the stream is closed.
          + */
          + @Override
          + public synchronized void close() {
          + if (byteBuffer != null)

          { + LOG.debug("releasing buffer"); + releaseBuffer(byteBuffer); + byteBuffer = null; + }

          + }
          +
          + /**
          + * Verify that the stream is open.
          + * @throws IOException if the stream is closed
          + */
          + private void verifyOpen() throws IOException {
          + if (byteBuffer == null)

          { + throw new IOException(FSExceptionMessages.STREAM_IS_CLOSED); + }

          + }
          +
          + public synchronized int read() throws IOException {
          + if (available() > 0)

          { + return byteBuffer.get() & 0xFF; + }

          else

          { + return -1; + }
          + }
          +
          + @Override
          + public synchronized long skip(long offset) throws IOException {
          + verifyOpen();
          + long newPos = position() + offset;
          + if (newPos < 0) { + throw new EOFException(FSExceptionMessages.NEGATIVE_SEEK); + }
          + if (newPos > size) { + throw new EOFException(FSExceptionMessages.CANNOT_SEEK_PAST_EOF); + }
          + byteBuffer.position((int) newPos);
          + return newPos;
          + }
          +
          + @Override
          + public synchronized int available() { + Preconditions.checkState(byteBuffer != null, + FSExceptionMessages.STREAM_IS_CLOSED); + return byteBuffer.remaining(); + }
          +
          + /**
          + * Get the current buffer position.
          + * @return the buffer position
          + */
          + public synchronized int position() { + return byteBuffer.position(); + }
          +
          + /**
          + * Check if there is data left.
          + * @return true if there is data remaining in the buffer.
          + */
          + public synchronized boolean hasRemaining() { + return byteBuffer.hasRemaining(); + }
          +
          + @Override
          + public synchronized void mark(int readlimit) {
          + LOG.debug("mark at {}", position());
          + byteBuffer.mark();
          + }
          +
          + @Override
          + public synchronized void reset() throws IOException { + LOG.debug("reset"); + byteBuffer.reset(); + }
          +
          + @Override
          + public boolean markSupported() { + return true; + }
          +
          + /**
          + * Read in data.
          + * @param buffer destination buffer
          + * @param offset offset within the buffer
          + * @param length length of bytes to read
          + * @throws EOFException if the position is negative
          + * @throws IndexOutOfBoundsException if there isn't space for the
          + * amount of data requested.
          + * @throws IllegalArgumentException other arguments are invalid.
          + */
          + @SuppressWarnings("NullableProblems")
          + public synchronized int read(byte[] buffer, int offset, int length)
          + throws IOException {
          + Preconditions.checkArgument(length >= 0, "length is negative");
          + Preconditions.checkArgument(buffer != null, "Null buffer");
          + if (buffer.length - offset < length) { + throw new IndexOutOfBoundsException( + FSExceptionMessages.TOO_MANY_BYTES_FOR_DEST_BUFFER + + ": request length =" + length + + ", with offset =" + offset + + "; buffer capacity =" + (buffer.length - offset)); + }
          + verifyOpen();
          + if (!hasRemaining()) { + return -1; + }

          +
          + int toRead = Math.min(length, available());
          + byteBuffer.get(buffer, offset, toRead);
          + return toRead;
          + }
          +
          + @Override
          + public String toString() {
          + final StringBuilder sb = new StringBuilder(
          + "ByteBufferInputStream{");
          + sb.append("size=").append(size);
          + ByteBuffer buffer = this.byteBuffer;
          + if (buffer != null)

          { + sb.append(", available=").append(buffer.remaining()); + }

          + sb.append('}');
          + return sb.toString();
          + }
          + }
          + }
          +
          + // ====================================================================
          +
          + /**
          + * Buffer blocks to disk.
          + */
          + static class DiskBlockFactory extends BlockFactory {
          +
          + DiskBlockFactory(S3AFileSystem owner)

          { + super(owner); + }

          +
          + /**
          + * Create a temp file and a block which writes to it.
          + * @param limit limit of the block.
          + * @return the new block
          + * @throws IOException IO problems
          + */
          + @Override
          + DataBlock create(int limit) throws IOException

          { + File destFile = owner + .createTmpFileForWrite("s3ablock", limit, owner.getConf()); + return new DiskBlock(destFile, limit); + }

          + }
          +
          + /**
          + * Stream to a file.
          + * This will stop at the limit; the caller is expected to create a new block
          + */
          + static class DiskBlock extends DataBlock {
          +
          + protected int bytesWritten;
          + private final File bufferFile;
          + private final int limit;
          + private BufferedOutputStream out;
          + private InputStream uploadStream;
          +
          + DiskBlock(File bufferFile, int limit)
          + throws FileNotFoundException

          { + this.limit = limit; + this.bufferFile = bufferFile; + out = new BufferedOutputStream(new FileOutputStream(bufferFile)); + }

          +
          + @Override
          + int dataSize()

          { + return bytesWritten; + }

          +
          + @Override
          + boolean hasCapacity(long bytes)

          { + return dataSize() + bytes <= limit; + }

          +
          + @Override
          + int remainingCapacity()

          { + return limit - bytesWritten; + }

          +
          + @Override
          + int write(byte[] b, int offset, int len) throws IOException

          { + super.write(b, offset, len); + int written = Math.min(remainingCapacity(), len); + out.write(b, offset, written); + bytesWritten += written; + return written; + }

          +
          + @Override
          + InputStream startUpload() throws IOException {
          + super.startUpload();
          + try

          { + out.flush(); + }

          finally

          { + out.close(); + out = null; + }

          + uploadStream = new FileInputStream(bufferFile);
          + return new FileDeletingInputStream(uploadStream);
          + }
          +
          + /**
          + * The close operation will delete the destination file if it still
          + * exists.
          + * @throws IOException IO problems
          + */
          + @Override
          + protected void innerClose() throws IOException {
          + final DestState state = getState();
          + LOG.debug("Closing {}", this);
          + switch (state) {
          + case Writing:
          + if (bufferFile.exists()) {
          + // file was not uploaded
          + LOG.debug("Deleting buffer file as upload did not start");
          + boolean deleted = bufferFile.delete();
          + if (!deleted && bufferFile.exists()) {
          + LOG.warn("Failed to delete buffer file {}", bufferFile);
          + }
          + }
          + break;
          +
          + case Upload:
          + LOG.debug("Buffer file {} exists —close upload stream", bufferFile);
          + break;
          +
          + case Closed:
          + // no-op
          + break;
          +
          + default:
          + // this state can never be reached, but checkstyle complains, so
          + // it is here.
          + }
          + }
          +
          + /**
          + * Flush operation will flush to disk.
          + * @throws IOException IOE raised on FileOutputStream
          + */
          + @Override
          + void flush() throws IOException {
          — End diff –

          Call `super.flush()` to trigger the validation check for `Writing` state.

          Show
          githubbot ASF GitHub Bot added a comment - Github user cnauroth commented on a diff in the pull request: https://github.com/apache/hadoop/pull/130#discussion_r82086784 — Diff: hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ADataBlocks.java — @@ -0,0 +1,816 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.s3a; + +import java.io.BufferedOutputStream; +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.Closeable; +import java.io.EOFException; +import java.io.File; +import java.io.FileInputStream; +import java.io.FileNotFoundException; +import java.io.FileOutputStream; +import java.io.FilterInputStream; +import java.io.IOException; +import java.io.InputStream; +import java.nio.BufferUnderflowException; +import java.nio.ByteBuffer; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; + +import com.google.common.base.Preconditions; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.fs.FSExceptionMessages; +import org.apache.hadoop.util.DirectBufferPool; + +import static org.apache.hadoop.fs.s3a.S3ADataBlocks.DataBlock.DestState.*; + +/** + * Set of classes to support output streaming into blocks which are then + * uploaded as partitions. + */ +final class S3ADataBlocks { + + private static final Logger LOG = LoggerFactory.getLogger(S3ADataBlocks.class); + + private S3ADataBlocks() { + } + + /** + * Validate args to a write command. These are the same validation checks + * expected for any implementation of {@code OutputStream.write()} . + * @param b byte array containing data + * @param off offset in array where to start + * @param len number of bytes to be written + * @throws NullPointerException for a null buffer + * @throws IndexOutOfBoundsException if indices are out of range + */ + static void validateWriteArgs(byte[] b, int off, int len) + throws IOException { + Preconditions.checkNotNull(b); + if ((off < 0) || (off > b.length) || (len < 0) || + ((off + len) > b.length) || ((off + len) < 0)) { + throw new IndexOutOfBoundsException( + "write (b[" + b.length + "], " + off + ", " + len + ')'); + } + } + + /** + * Create a factory. + * @param owner factory owner + * @param name factory name -the option from {@link Constants} . + * @return the factory, ready to be initialized. + * @throws IllegalArgumentException if the name is unknown. + */ + static BlockFactory createFactory(S3AFileSystem owner, + String name) { + switch (name) { + case Constants.BLOCK_OUTPUT_BUFFER_ARRAY: + return new ArrayBlockFactory(owner); + case Constants.BLOCK_OUTPUT_BUFFER_DISK: + return new DiskBlockFactory(owner); + case Constants.BLOCK_OUTPUT_BYTEBUFFER: + return new ByteBufferBlockFactory(owner); + default: + throw new IllegalArgumentException("Unsupported block buffer" + + " \"" + name + '"'); + } + } + + /** + * Base class for block factories. + */ + static abstract class BlockFactory implements Closeable { + + /** + * Owner. + */ + protected final S3AFileSystem owner; + + protected BlockFactory(S3AFileSystem owner) { + this.owner = owner; + } + + /** + * Create a block. + * @param limit limit of the block. + * @return a new block. + */ + abstract DataBlock create(int limit) throws IOException; + + /** + * Implement any close/cleanup operation. + * Base class is a no-op + * @throws IOException -ideally, it shouldn't. + */ + @Override + public void close() throws IOException { + } + } + + /** + * This represents a block being uploaded. + */ + static abstract class DataBlock implements Closeable { + + private volatile DestState state = Writing; + + /** + * Enter + * @param current + * @param next + * @throws IllegalStateException + */ + protected synchronized final void enterState(DestState current, + DestState next) + throws IllegalStateException { + verifyState(current); + LOG.debug("{}: entering state {}", this, next); + state = next; + } + + /** + * Verify that the block is in the declared state. + * @param expected expected state. + * @throws IllegalStateException if the DataBlock is in the wrong state + */ + protected final void verifyState(DestState expected) throws IllegalStateException { + if (expected != null && state != expected) { + throw new IllegalStateException("Expected stream state " + expected + + " -but actual state is " + state + " in " + this); + } + } + + /** + * Current state. + * @return the current state. + */ + final DestState getState() { + return state; + } + + /** + * Return the current data size. + * @return the size of the data + */ + abstract int dataSize(); + + /** + * Predicate to verify that the block has the capacity to write + * the given set of bytes. + * @param bytes number of bytes desired to be written. + * @return true if there is enough space. + */ + abstract boolean hasCapacity(long bytes); + + /** + * Predicate to check if there is data in the block. + * @return true if there is + */ + boolean hasData() { + return dataSize() > 0; + } + + /** + * The remaining capacity in the block before it is full. + * @return the number of bytes remaining. + */ + abstract int remainingCapacity(); + + /** + * Write a series of bytes from the buffer, from the offset. + * Returns the number of bytes written. + * Only valid in the state {@code Writing}. + * Base class verifies the state but does no writing. + * @param buffer buffer + * @param offset offset + * @param length length of write + * @return number of bytes written + * @throws IOException trouble + */ + int write(byte[] buffer, int offset, int length) throws IOException { + verifyState(Writing); + Preconditions.checkArgument(buffer != null, "Null buffer"); + Preconditions.checkArgument(length >= 0, "length is negative"); + Preconditions.checkArgument(offset >= 0, "offset is negative"); + Preconditions.checkArgument( + !(buffer.length - offset < length), + "buffer shorter than amount of data to write"); + return 0; + } + + /** + * Flush the output. + * Only valid in the state {@code Writing} . + * In the base class, this is a no-op + * @throws IOException any IO problem. + */ + void flush() throws IOException { + verifyState(Writing); + } + + /** + * Switch to the upload state and return a stream for uploading. + * Base class calls {@link #enterState(DestState, DestState)} to + * manage the state machine. + * @return the stream + * @throws IOException trouble + */ + InputStream startUpload() throws IOException { + LOG.debug("Start datablock upload"); + enterState(Writing, Upload); + return null; + } + + /** + * Enter the closed state. + * @return true if the class was in any other state, implying that + * the subclass should do its close operations + */ + protected synchronized boolean enterClosedState() { + if (!state.equals(Closed)) { + try { + enterState(null, Closed); + } catch (IllegalStateException ignored) { + + } + return true; + } else { + return false; + } + } + + @Override + public void close() throws IOException { + if (enterClosedState()) { + LOG.debug("Closed {}", this); + innerClose(); + } + } + + /** + * Inner close logic for subclasses to implement. + */ + protected void innerClose() throws IOException { + + } + + enum DestState {Writing, Upload, Closed} + } + + // ==================================================================== + + /** + * Use byte arrays on the heap for storage. + */ + static class ArrayBlockFactory extends BlockFactory { + + ArrayBlockFactory(S3AFileSystem owner) { + super(owner); + } + + @Override + DataBlock create(int limit) throws IOException { + return new ByteArrayBlock(limit); + } + + } + + /** + * Stream to memory via a {@code ByteArrayOutputStream}. + * + * This was taken from {@code S3AFastOutputStream} and has the + * same problem which surfaced there: it consumes heap space + * proportional to the mismatch between writes to the stream and + * the JVM-wide upload bandwidth to the S3 endpoint. + */ + + static class ByteArrayBlock extends DataBlock { + private ByteArrayOutputStream buffer; + private final int limit; + // cache data size so that it is consistent after the buffer is reset. + private Integer dataSize; + + ByteArrayBlock(int limit) { + this.limit = limit; + buffer = new ByteArrayOutputStream(); + } + + /** + * Get the amount of data; if there is no buffer then the size is 0. + * @return the amount of data available to upload. + */ + @Override + int dataSize() { + return dataSize != null ? dataSize : buffer.size(); + } + + @Override + InputStream startUpload() throws IOException { + super.startUpload(); + dataSize = buffer.size(); + ByteArrayInputStream bufferData = new ByteArrayInputStream( + buffer.toByteArray()); + buffer.reset(); + buffer = null; + return bufferData; + } + + @Override + boolean hasCapacity(long bytes) { + return dataSize() + bytes <= limit; + } + + @Override + int remainingCapacity() { + return limit - dataSize(); + } + + @Override + int write(byte[] b, int offset, int len) throws IOException { + super.write(b, offset, len); + int written = Math.min(remainingCapacity(), len); + buffer.write(b, offset, written); + return written; + } + + @Override + protected void innerClose() { + buffer = null; + } + + @Override + public String toString() { + return "ByteArrayBlock{" + + "state=" + getState() + + ", limit=" + limit + + ", dataSize=" + dataSize + + '}'; + } + } + + // ==================================================================== + + /** + * Stream via Direct ByteBuffers; these are allocated off heap + * via {@link DirectBufferPool}. + * This is actually the most complex of all the block factories, + * due to the need to explicitly recycle buffers; in comparison, the + * {@link DiskBlock} buffer delegates the work of deleting files to + * the {@link DiskBlock.FileDeletingInputStream}. Here the + * input stream {@link ByteBufferInputStream} has a similar task, along + * with the foundational work of streaming data from a byte array. + */ + + static class ByteBufferBlockFactory extends BlockFactory { + + private final DirectBufferPool bufferPool = new DirectBufferPool(); + private final AtomicInteger buffersOutstanding = new AtomicInteger(0); + + ByteBufferBlockFactory(S3AFileSystem owner) { + super(owner); + } + + @Override + ByteBufferBlock create(int limit) throws IOException { + return new ByteBufferBlock(limit); + } + + private ByteBuffer requestBuffer(int limit) { + LOG.debug("Requesting buffer of size {}", limit); + buffersOutstanding.incrementAndGet(); + return bufferPool.getBuffer(limit); + } + + private void releaseBuffer(ByteBuffer buffer) { + LOG.debug("Releasing buffer"); + bufferPool.returnBuffer(buffer); + buffersOutstanding.decrementAndGet(); + } + + /** + * Get count of outstanding buffers. + * @return the current buffer count + */ + public int getOutstandingBufferCount() { + return buffersOutstanding.get(); + } + + @Override + public String toString() { + return "ByteBufferBlockFactory {" + + "buffersOutstanding=" + buffersOutstanding + + '} '; + } + + /** + * A DataBlock which requests a buffer from pool on creation; returns + * it when the output stream is closed. + */ + class ByteBufferBlock extends DataBlock { + private ByteBuffer buffer; + private final int bufferSize; + // cache data size so that it is consistent after the buffer is reset. + private Integer dataSize; + + /** + * Instantiate. This will request a ByteBuffer of the desired size. + * @param bufferSize buffer size + */ + ByteBufferBlock(int bufferSize) { + this.bufferSize = bufferSize; + buffer = requestBuffer(bufferSize); + } + + /** + * Get the amount of data; if there is no buffer then the size is 0. + * @return the amount of data available to upload. + */ + @Override + int dataSize() { + return dataSize != null ? dataSize : bufferCapacityUsed(); + } + + @Override + ByteBufferInputStream startUpload() throws IOException { + super.startUpload(); + dataSize = bufferCapacityUsed(); + // set the buffer up from reading from the beginning + buffer.limit(buffer.position()); + buffer.position(0); + return new ByteBufferInputStream(dataSize, buffer); + } + + @Override + public boolean hasCapacity(long bytes) { + return bytes <= remainingCapacity(); + } + + @Override + public int remainingCapacity() { + return buffer != null ? buffer.remaining() : 0; + } + + private int bufferCapacityUsed() { + return buffer.capacity() - buffer.remaining(); + } + + @Override + int write(byte[] b, int offset, int len) throws IOException { + super.write(b, offset, len); + int written = Math.min(remainingCapacity(), len); + buffer.put(b, offset, written); + return written; + } + + @Override + protected void innerClose() { + buffer = null; + } + + @Override + public String toString() { + return "ByteBufferBlock {" + + "state=" + getState() + + ", dataSize=" + dataSize() + + ", limit=" + bufferSize + + ", remainingCapacity=" + remainingCapacity() + + '} '; + } + + } + + /** + * Provide an input stream from a byte buffer; supporting + * {@link #mark(int)} , which is required to enable replay of failed + * PUT attempts. + * This input stream returns the buffer to the pool afterwards. + */ + class ByteBufferInputStream extends InputStream { + + private final int size; + private ByteBuffer byteBuffer; + + ByteBufferInputStream(int size, ByteBuffer byteBuffer) { + LOG.debug("Creating ByteBufferInputStream of size {}", size); + this.size = size; + this.byteBuffer = byteBuffer; + } + + /** + * Return the buffer to the pool after the stream is closed. + */ + @Override + public synchronized void close() { + if (byteBuffer != null) { + LOG.debug("releasing buffer"); + releaseBuffer(byteBuffer); + byteBuffer = null; + } + } + + /** + * Verify that the stream is open. + * @throws IOException if the stream is closed + */ + private void verifyOpen() throws IOException { + if (byteBuffer == null) { + throw new IOException(FSExceptionMessages.STREAM_IS_CLOSED); + } + } + + public synchronized int read() throws IOException { + if (available() > 0) { + return byteBuffer.get() & 0xFF; + } else { + return -1; + } + } + + @Override + public synchronized long skip(long offset) throws IOException { + verifyOpen(); + long newPos = position() + offset; + if (newPos < 0) { + throw new EOFException(FSExceptionMessages.NEGATIVE_SEEK); + } + if (newPos > size) { + throw new EOFException(FSExceptionMessages.CANNOT_SEEK_PAST_EOF); + } + byteBuffer.position((int) newPos); + return newPos; + } + + @Override + public synchronized int available() { + Preconditions.checkState(byteBuffer != null, + FSExceptionMessages.STREAM_IS_CLOSED); + return byteBuffer.remaining(); + } + + /** + * Get the current buffer position. + * @return the buffer position + */ + public synchronized int position() { + return byteBuffer.position(); + } + + /** + * Check if there is data left. + * @return true if there is data remaining in the buffer. + */ + public synchronized boolean hasRemaining() { + return byteBuffer.hasRemaining(); + } + + @Override + public synchronized void mark(int readlimit) { + LOG.debug("mark at {}", position()); + byteBuffer.mark(); + } + + @Override + public synchronized void reset() throws IOException { + LOG.debug("reset"); + byteBuffer.reset(); + } + + @Override + public boolean markSupported() { + return true; + } + + /** + * Read in data. + * @param buffer destination buffer + * @param offset offset within the buffer + * @param length length of bytes to read + * @throws EOFException if the position is negative + * @throws IndexOutOfBoundsException if there isn't space for the + * amount of data requested. + * @throws IllegalArgumentException other arguments are invalid. + */ + @SuppressWarnings("NullableProblems") + public synchronized int read(byte[] buffer, int offset, int length) + throws IOException { + Preconditions.checkArgument(length >= 0, "length is negative"); + Preconditions.checkArgument(buffer != null, "Null buffer"); + if (buffer.length - offset < length) { + throw new IndexOutOfBoundsException( + FSExceptionMessages.TOO_MANY_BYTES_FOR_DEST_BUFFER + + ": request length =" + length + + ", with offset =" + offset + + "; buffer capacity =" + (buffer.length - offset)); + } + verifyOpen(); + if (!hasRemaining()) { + return -1; + } + + int toRead = Math.min(length, available()); + byteBuffer.get(buffer, offset, toRead); + return toRead; + } + + @Override + public String toString() { + final StringBuilder sb = new StringBuilder( + "ByteBufferInputStream{"); + sb.append("size=").append(size); + ByteBuffer buffer = this.byteBuffer; + if (buffer != null) { + sb.append(", available=").append(buffer.remaining()); + } + sb.append('}'); + return sb.toString(); + } + } + } + + // ==================================================================== + + /** + * Buffer blocks to disk. + */ + static class DiskBlockFactory extends BlockFactory { + + DiskBlockFactory(S3AFileSystem owner) { + super(owner); + } + + /** + * Create a temp file and a block which writes to it. + * @param limit limit of the block. + * @return the new block + * @throws IOException IO problems + */ + @Override + DataBlock create(int limit) throws IOException { + File destFile = owner + .createTmpFileForWrite("s3ablock", limit, owner.getConf()); + return new DiskBlock(destFile, limit); + } + } + + /** + * Stream to a file. + * This will stop at the limit; the caller is expected to create a new block + */ + static class DiskBlock extends DataBlock { + + protected int bytesWritten; + private final File bufferFile; + private final int limit; + private BufferedOutputStream out; + private InputStream uploadStream; + + DiskBlock(File bufferFile, int limit) + throws FileNotFoundException { + this.limit = limit; + this.bufferFile = bufferFile; + out = new BufferedOutputStream(new FileOutputStream(bufferFile)); + } + + @Override + int dataSize() { + return bytesWritten; + } + + @Override + boolean hasCapacity(long bytes) { + return dataSize() + bytes <= limit; + } + + @Override + int remainingCapacity() { + return limit - bytesWritten; + } + + @Override + int write(byte[] b, int offset, int len) throws IOException { + super.write(b, offset, len); + int written = Math.min(remainingCapacity(), len); + out.write(b, offset, written); + bytesWritten += written; + return written; + } + + @Override + InputStream startUpload() throws IOException { + super.startUpload(); + try { + out.flush(); + } finally { + out.close(); + out = null; + } + uploadStream = new FileInputStream(bufferFile); + return new FileDeletingInputStream(uploadStream); + } + + /** + * The close operation will delete the destination file if it still + * exists. + * @throws IOException IO problems + */ + @Override + protected void innerClose() throws IOException { + final DestState state = getState(); + LOG.debug("Closing {}", this); + switch (state) { + case Writing: + if (bufferFile.exists()) { + // file was not uploaded + LOG.debug("Deleting buffer file as upload did not start"); + boolean deleted = bufferFile.delete(); + if (!deleted && bufferFile.exists()) { + LOG.warn("Failed to delete buffer file {}", bufferFile); + } + } + break; + + case Upload: + LOG.debug("Buffer file {} exists —close upload stream", bufferFile); + break; + + case Closed: + // no-op + break; + + default: + // this state can never be reached, but checkstyle complains, so + // it is here. + } + } + + /** + * Flush operation will flush to disk. + * @throws IOException IOE raised on FileOutputStream + */ + @Override + void flush() throws IOException { — End diff – Call `super.flush()` to trigger the validation check for `Writing` state.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user cnauroth commented on a diff in the pull request:

          https://github.com/apache/hadoop/pull/130#discussion_r82086168

          — Diff: hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ADataBlocks.java —
          @@ -0,0 +1,816 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +
          +package org.apache.hadoop.fs.s3a;
          +
          +import java.io.BufferedOutputStream;
          +import java.io.ByteArrayInputStream;
          +import java.io.ByteArrayOutputStream;
          +import java.io.Closeable;
          +import java.io.EOFException;
          +import java.io.File;
          +import java.io.FileInputStream;
          +import java.io.FileNotFoundException;
          +import java.io.FileOutputStream;
          +import java.io.FilterInputStream;
          +import java.io.IOException;
          +import java.io.InputStream;
          +import java.nio.BufferUnderflowException;
          +import java.nio.ByteBuffer;
          +import java.util.concurrent.atomic.AtomicBoolean;
          +import java.util.concurrent.atomic.AtomicInteger;
          +
          +import com.google.common.base.Preconditions;
          +import org.slf4j.Logger;
          +import org.slf4j.LoggerFactory;
          +
          +import org.apache.hadoop.fs.FSExceptionMessages;
          +import org.apache.hadoop.util.DirectBufferPool;
          +
          +import static org.apache.hadoop.fs.s3a.S3ADataBlocks.DataBlock.DestState.*;
          +
          +/**
          + * Set of classes to support output streaming into blocks which are then
          + * uploaded as partitions.
          + */
          +final class S3ADataBlocks {
          +
          + private static final Logger LOG = LoggerFactory.getLogger(S3ADataBlocks.class);
          +
          + private S3ADataBlocks()

          { + }

          +
          + /**
          + * Validate args to a write command. These are the same validation checks
          + * expected for any implementation of

          {@code OutputStream.write()}

          .
          + * @param b byte array containing data
          + * @param off offset in array where to start
          + * @param len number of bytes to be written
          + * @throws NullPointerException for a null buffer
          + * @throws IndexOutOfBoundsException if indices are out of range
          + */
          + static void validateWriteArgs(byte[] b, int off, int len)
          + throws IOException {
          + Preconditions.checkNotNull(b);
          + if ((off < 0) || (off > b.length) || (len < 0) ||
          + ((off + len) > b.length) || ((off + len) < 0))

          { + throw new IndexOutOfBoundsException( + "write (b[" + b.length + "], " + off + ", " + len + ')'); + }

          + }
          +
          + /**
          + * Create a factory.
          + * @param owner factory owner
          + * @param name factory name -the option from

          {@link Constants}

          .
          + * @return the factory, ready to be initialized.
          + * @throws IllegalArgumentException if the name is unknown.
          + */
          + static BlockFactory createFactory(S3AFileSystem owner,
          + String name) {
          + switch (name)

          { + case Constants.BLOCK_OUTPUT_BUFFER_ARRAY: + return new ArrayBlockFactory(owner); + case Constants.BLOCK_OUTPUT_BUFFER_DISK: + return new DiskBlockFactory(owner); + case Constants.BLOCK_OUTPUT_BYTEBUFFER: + return new ByteBufferBlockFactory(owner); + default: + throw new IllegalArgumentException("Unsupported block buffer" + + " \"" + name + '"'); + }

          + }
          +
          + /**
          + * Base class for block factories.
          + */
          + static abstract class BlockFactory implements Closeable {
          +
          + /**
          + * Owner.
          + */
          + protected final S3AFileSystem owner;
          +
          + protected BlockFactory(S3AFileSystem owner)

          { + this.owner = owner; + }

          +
          + /**
          + * Create a block.
          + * @param limit limit of the block.
          + * @return a new block.
          + */
          + abstract DataBlock create(int limit) throws IOException;
          +
          + /**
          + * Implement any close/cleanup operation.
          + * Base class is a no-op
          + * @throws IOException -ideally, it shouldn't.
          + */
          + @Override
          + public void close() throws IOException

          { + }

          + }
          +
          + /**
          + * This represents a block being uploaded.
          + */
          + static abstract class DataBlock implements Closeable {
          +
          + private volatile DestState state = Writing;
          +
          + /**
          + * Enter
          + * @param current
          + * @param next
          + * @throws IllegalStateException
          + */
          + protected synchronized final void enterState(DestState current,
          + DestState next)
          + throws IllegalStateException {
          + verifyState(current);
          + LOG.debug("{}: entering state {}", this, next);
          + state = next;
          + }
          +
          + /**
          + * Verify that the block is in the declared state.
          + * @param expected expected state.
          + * @throws IllegalStateException if the DataBlock is in the wrong state
          + */
          + protected final void verifyState(DestState expected) throws IllegalStateException {
          + if (expected != null && state != expected)

          { + throw new IllegalStateException("Expected stream state " + expected + + " -but actual state is " + state + " in " + this); + }

          + }
          +
          + /**
          + * Current state.
          + * @return the current state.
          + */
          + final DestState getState()

          { + return state; + }

          +
          + /**
          + * Return the current data size.
          + * @return the size of the data
          + */
          + abstract int dataSize();
          +
          + /**
          + * Predicate to verify that the block has the capacity to write
          + * the given set of bytes.
          + * @param bytes number of bytes desired to be written.
          + * @return true if there is enough space.
          + */
          + abstract boolean hasCapacity(long bytes);
          +
          + /**
          + * Predicate to check if there is data in the block.
          + * @return true if there is
          + */
          + boolean hasData()

          { + return dataSize() > 0; + }

          +
          + /**
          + * The remaining capacity in the block before it is full.
          + * @return the number of bytes remaining.
          + */
          + abstract int remainingCapacity();
          +
          + /**
          + * Write a series of bytes from the buffer, from the offset.
          + * Returns the number of bytes written.
          + * Only valid in the state

          {@code Writing}.
          + * Base class verifies the state but does no writing.
          + * @param buffer buffer
          + * @param offset offset
          + * @param length length of write
          + * @return number of bytes written
          + * @throws IOException trouble
          + */
          + int write(byte[] buffer, int offset, int length) throws IOException { + verifyState(Writing); + Preconditions.checkArgument(buffer != null, "Null buffer"); + Preconditions.checkArgument(length >= 0, "length is negative"); + Preconditions.checkArgument(offset >= 0, "offset is negative"); + Preconditions.checkArgument( + !(buffer.length - offset < length), + "buffer shorter than amount of data to write"); + return 0; + }
          +
          + /**
          + * Flush the output.
          + * Only valid in the state {@code Writing}

          .
          + * In the base class, this is a no-op
          + * @throws IOException any IO problem.
          + */
          + void flush() throws IOException

          { + verifyState(Writing); + }

          +
          + /**
          + * Switch to the upload state and return a stream for uploading.
          + * Base class calls

          {@link #enterState(DestState, DestState)}

          to
          + * manage the state machine.
          + * @return the stream
          + * @throws IOException trouble
          + */
          + InputStream startUpload() throws IOException

          { + LOG.debug("Start datablock upload"); + enterState(Writing, Upload); + return null; + }

          +
          + /**
          + * Enter the closed state.
          + * @return true if the class was in any other state, implying that
          + * the subclass should do its close operations
          + */
          + protected synchronized boolean enterClosedState() {
          + if (!state.equals(Closed)) {
          + try

          { + enterState(null, Closed); + }

          catch (IllegalStateException ignored) {
          — End diff –

          If I understand correctly, this can't throw the exception unless we have a bug in our code. Is it better to let the `IllegalStateException` be thrown so that we see that sooner?

          Show
          githubbot ASF GitHub Bot added a comment - Github user cnauroth commented on a diff in the pull request: https://github.com/apache/hadoop/pull/130#discussion_r82086168 — Diff: hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ADataBlocks.java — @@ -0,0 +1,816 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.s3a; + +import java.io.BufferedOutputStream; +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.Closeable; +import java.io.EOFException; +import java.io.File; +import java.io.FileInputStream; +import java.io.FileNotFoundException; +import java.io.FileOutputStream; +import java.io.FilterInputStream; +import java.io.IOException; +import java.io.InputStream; +import java.nio.BufferUnderflowException; +import java.nio.ByteBuffer; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; + +import com.google.common.base.Preconditions; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.fs.FSExceptionMessages; +import org.apache.hadoop.util.DirectBufferPool; + +import static org.apache.hadoop.fs.s3a.S3ADataBlocks.DataBlock.DestState.*; + +/** + * Set of classes to support output streaming into blocks which are then + * uploaded as partitions. + */ +final class S3ADataBlocks { + + private static final Logger LOG = LoggerFactory.getLogger(S3ADataBlocks.class); + + private S3ADataBlocks() { + } + + /** + * Validate args to a write command. These are the same validation checks + * expected for any implementation of {@code OutputStream.write()} . + * @param b byte array containing data + * @param off offset in array where to start + * @param len number of bytes to be written + * @throws NullPointerException for a null buffer + * @throws IndexOutOfBoundsException if indices are out of range + */ + static void validateWriteArgs(byte[] b, int off, int len) + throws IOException { + Preconditions.checkNotNull(b); + if ((off < 0) || (off > b.length) || (len < 0) || + ((off + len) > b.length) || ((off + len) < 0)) { + throw new IndexOutOfBoundsException( + "write (b[" + b.length + "], " + off + ", " + len + ')'); + } + } + + /** + * Create a factory. + * @param owner factory owner + * @param name factory name -the option from {@link Constants} . + * @return the factory, ready to be initialized. + * @throws IllegalArgumentException if the name is unknown. + */ + static BlockFactory createFactory(S3AFileSystem owner, + String name) { + switch (name) { + case Constants.BLOCK_OUTPUT_BUFFER_ARRAY: + return new ArrayBlockFactory(owner); + case Constants.BLOCK_OUTPUT_BUFFER_DISK: + return new DiskBlockFactory(owner); + case Constants.BLOCK_OUTPUT_BYTEBUFFER: + return new ByteBufferBlockFactory(owner); + default: + throw new IllegalArgumentException("Unsupported block buffer" + + " \"" + name + '"'); + } + } + + /** + * Base class for block factories. + */ + static abstract class BlockFactory implements Closeable { + + /** + * Owner. + */ + protected final S3AFileSystem owner; + + protected BlockFactory(S3AFileSystem owner) { + this.owner = owner; + } + + /** + * Create a block. + * @param limit limit of the block. + * @return a new block. + */ + abstract DataBlock create(int limit) throws IOException; + + /** + * Implement any close/cleanup operation. + * Base class is a no-op + * @throws IOException -ideally, it shouldn't. + */ + @Override + public void close() throws IOException { + } + } + + /** + * This represents a block being uploaded. + */ + static abstract class DataBlock implements Closeable { + + private volatile DestState state = Writing; + + /** + * Enter + * @param current + * @param next + * @throws IllegalStateException + */ + protected synchronized final void enterState(DestState current, + DestState next) + throws IllegalStateException { + verifyState(current); + LOG.debug("{}: entering state {}", this, next); + state = next; + } + + /** + * Verify that the block is in the declared state. + * @param expected expected state. + * @throws IllegalStateException if the DataBlock is in the wrong state + */ + protected final void verifyState(DestState expected) throws IllegalStateException { + if (expected != null && state != expected) { + throw new IllegalStateException("Expected stream state " + expected + + " -but actual state is " + state + " in " + this); + } + } + + /** + * Current state. + * @return the current state. + */ + final DestState getState() { + return state; + } + + /** + * Return the current data size. + * @return the size of the data + */ + abstract int dataSize(); + + /** + * Predicate to verify that the block has the capacity to write + * the given set of bytes. + * @param bytes number of bytes desired to be written. + * @return true if there is enough space. + */ + abstract boolean hasCapacity(long bytes); + + /** + * Predicate to check if there is data in the block. + * @return true if there is + */ + boolean hasData() { + return dataSize() > 0; + } + + /** + * The remaining capacity in the block before it is full. + * @return the number of bytes remaining. + */ + abstract int remainingCapacity(); + + /** + * Write a series of bytes from the buffer, from the offset. + * Returns the number of bytes written. + * Only valid in the state {@code Writing}. + * Base class verifies the state but does no writing. + * @param buffer buffer + * @param offset offset + * @param length length of write + * @return number of bytes written + * @throws IOException trouble + */ + int write(byte[] buffer, int offset, int length) throws IOException { + verifyState(Writing); + Preconditions.checkArgument(buffer != null, "Null buffer"); + Preconditions.checkArgument(length >= 0, "length is negative"); + Preconditions.checkArgument(offset >= 0, "offset is negative"); + Preconditions.checkArgument( + !(buffer.length - offset < length), + "buffer shorter than amount of data to write"); + return 0; + } + + /** + * Flush the output. + * Only valid in the state {@code Writing} . + * In the base class, this is a no-op + * @throws IOException any IO problem. + */ + void flush() throws IOException { + verifyState(Writing); + } + + /** + * Switch to the upload state and return a stream for uploading. + * Base class calls {@link #enterState(DestState, DestState)} to + * manage the state machine. + * @return the stream + * @throws IOException trouble + */ + InputStream startUpload() throws IOException { + LOG.debug("Start datablock upload"); + enterState(Writing, Upload); + return null; + } + + /** + * Enter the closed state. + * @return true if the class was in any other state, implying that + * the subclass should do its close operations + */ + protected synchronized boolean enterClosedState() { + if (!state.equals(Closed)) { + try { + enterState(null, Closed); + } catch (IllegalStateException ignored) { — End diff – If I understand correctly, this can't throw the exception unless we have a bug in our code. Is it better to let the `IllegalStateException` be thrown so that we see that sooner?
          Hide
          cnauroth Chris Nauroth added a comment -

          Steve, thank you for the update. I have entered a few more small comments on pull request #130. I think this is close to being ready after addressing those comments and getting another pre-commit run.

          Show
          cnauroth Chris Nauroth added a comment - Steve, thank you for the update. I have entered a few more small comments on pull request #130. I think this is close to being ready after addressing those comments and getting another pre-commit run.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user steveloughran commented on a diff in the pull request:

          https://github.com/apache/hadoop/pull/130#discussion_r82171213

          — Diff: hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ADataBlocks.java —
          @@ -0,0 +1,816 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +
          +package org.apache.hadoop.fs.s3a;
          +
          +import java.io.BufferedOutputStream;
          +import java.io.ByteArrayInputStream;
          +import java.io.ByteArrayOutputStream;
          +import java.io.Closeable;
          +import java.io.EOFException;
          +import java.io.File;
          +import java.io.FileInputStream;
          +import java.io.FileNotFoundException;
          +import java.io.FileOutputStream;
          +import java.io.FilterInputStream;
          +import java.io.IOException;
          +import java.io.InputStream;
          +import java.nio.BufferUnderflowException;
          +import java.nio.ByteBuffer;
          +import java.util.concurrent.atomic.AtomicBoolean;
          +import java.util.concurrent.atomic.AtomicInteger;
          +
          +import com.google.common.base.Preconditions;
          +import org.slf4j.Logger;
          +import org.slf4j.LoggerFactory;
          +
          +import org.apache.hadoop.fs.FSExceptionMessages;
          +import org.apache.hadoop.util.DirectBufferPool;
          +
          +import static org.apache.hadoop.fs.s3a.S3ADataBlocks.DataBlock.DestState.*;
          +
          +/**
          + * Set of classes to support output streaming into blocks which are then
          + * uploaded as partitions.
          + */
          +final class S3ADataBlocks {
          +
          + private static final Logger LOG = LoggerFactory.getLogger(S3ADataBlocks.class);
          +
          + private S3ADataBlocks()

          { + }

          +
          + /**
          + * Validate args to a write command. These are the same validation checks
          + * expected for any implementation of

          {@code OutputStream.write()}

          .
          + * @param b byte array containing data
          + * @param off offset in array where to start
          + * @param len number of bytes to be written
          + * @throws NullPointerException for a null buffer
          + * @throws IndexOutOfBoundsException if indices are out of range
          + */
          + static void validateWriteArgs(byte[] b, int off, int len)
          + throws IOException {
          + Preconditions.checkNotNull(b);
          + if ((off < 0) || (off > b.length) || (len < 0) ||
          + ((off + len) > b.length) || ((off + len) < 0))

          { + throw new IndexOutOfBoundsException( + "write (b[" + b.length + "], " + off + ", " + len + ')'); + }

          + }
          +
          + /**
          + * Create a factory.
          + * @param owner factory owner
          + * @param name factory name -the option from

          {@link Constants}

          .
          + * @return the factory, ready to be initialized.
          + * @throws IllegalArgumentException if the name is unknown.
          + */
          + static BlockFactory createFactory(S3AFileSystem owner,
          + String name) {
          + switch (name)

          { + case Constants.BLOCK_OUTPUT_BUFFER_ARRAY: + return new ArrayBlockFactory(owner); + case Constants.BLOCK_OUTPUT_BUFFER_DISK: + return new DiskBlockFactory(owner); + case Constants.BLOCK_OUTPUT_BYTEBUFFER: + return new ByteBufferBlockFactory(owner); + default: + throw new IllegalArgumentException("Unsupported block buffer" + + " \"" + name + '"'); + }

          + }
          +
          + /**
          + * Base class for block factories.
          + */
          + static abstract class BlockFactory implements Closeable {
          +
          + /**
          + * Owner.
          + */
          + protected final S3AFileSystem owner;
          +
          + protected BlockFactory(S3AFileSystem owner)

          { + this.owner = owner; + }

          +
          + /**
          + * Create a block.
          + * @param limit limit of the block.
          + * @return a new block.
          + */
          + abstract DataBlock create(int limit) throws IOException;
          +
          + /**
          + * Implement any close/cleanup operation.
          + * Base class is a no-op
          + * @throws IOException -ideally, it shouldn't.
          + */
          + @Override
          + public void close() throws IOException

          { + }

          + }
          +
          + /**
          + * This represents a block being uploaded.
          + */
          + static abstract class DataBlock implements Closeable {
          +
          + private volatile DestState state = Writing;
          +
          + /**
          + * Enter
          + * @param current
          + * @param next
          + * @throws IllegalStateException
          + */
          + protected synchronized final void enterState(DestState current,
          + DestState next)
          + throws IllegalStateException {
          + verifyState(current);
          + LOG.debug("{}: entering state {}", this, next);
          + state = next;
          + }
          +
          + /**
          + * Verify that the block is in the declared state.
          + * @param expected expected state.
          + * @throws IllegalStateException if the DataBlock is in the wrong state
          + */
          + protected final void verifyState(DestState expected) throws IllegalStateException {
          + if (expected != null && state != expected)

          { + throw new IllegalStateException("Expected stream state " + expected + + " -but actual state is " + state + " in " + this); + }

          + }
          +
          + /**
          + * Current state.
          + * @return the current state.
          + */
          + final DestState getState()

          { + return state; + }

          +
          + /**
          + * Return the current data size.
          + * @return the size of the data
          + */
          + abstract int dataSize();
          +
          + /**
          + * Predicate to verify that the block has the capacity to write
          + * the given set of bytes.
          + * @param bytes number of bytes desired to be written.
          + * @return true if there is enough space.
          + */
          + abstract boolean hasCapacity(long bytes);
          +
          + /**
          + * Predicate to check if there is data in the block.
          + * @return true if there is
          + */
          + boolean hasData()

          { + return dataSize() > 0; + }

          +
          + /**
          + * The remaining capacity in the block before it is full.
          + * @return the number of bytes remaining.
          + */
          + abstract int remainingCapacity();
          +
          + /**
          + * Write a series of bytes from the buffer, from the offset.
          + * Returns the number of bytes written.
          + * Only valid in the state

          {@code Writing}.
          + * Base class verifies the state but does no writing.
          + * @param buffer buffer
          + * @param offset offset
          + * @param length length of write
          + * @return number of bytes written
          + * @throws IOException trouble
          + */
          + int write(byte[] buffer, int offset, int length) throws IOException { + verifyState(Writing); + Preconditions.checkArgument(buffer != null, "Null buffer"); + Preconditions.checkArgument(length >= 0, "length is negative"); + Preconditions.checkArgument(offset >= 0, "offset is negative"); + Preconditions.checkArgument( + !(buffer.length - offset < length), + "buffer shorter than amount of data to write"); + return 0; + }
          +
          + /**
          + * Flush the output.
          + * Only valid in the state {@code Writing}

          .
          + * In the base class, this is a no-op
          + * @throws IOException any IO problem.
          + */
          + void flush() throws IOException

          { + verifyState(Writing); + }

          +
          + /**
          + * Switch to the upload state and return a stream for uploading.
          + * Base class calls

          {@link #enterState(DestState, DestState)}

          to
          + * manage the state machine.
          + * @return the stream
          + * @throws IOException trouble
          + */
          + InputStream startUpload() throws IOException

          { + LOG.debug("Start datablock upload"); + enterState(Writing, Upload); + return null; + }

          +
          + /**
          + * Enter the closed state.
          + * @return true if the class was in any other state, implying that
          + * the subclass should do its close operations
          + */
          + protected synchronized boolean enterClosedState() {
          + if (!state.equals(Closed)) {
          + try

          { + enterState(null, Closed); + }

          catch (IllegalStateException ignored) {
          — End diff –

          I know it can't happen, but like to close off all failure routes of a close() call. Anyway, throwing again.

          Show
          githubbot ASF GitHub Bot added a comment - Github user steveloughran commented on a diff in the pull request: https://github.com/apache/hadoop/pull/130#discussion_r82171213 — Diff: hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ADataBlocks.java — @@ -0,0 +1,816 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.s3a; + +import java.io.BufferedOutputStream; +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.Closeable; +import java.io.EOFException; +import java.io.File; +import java.io.FileInputStream; +import java.io.FileNotFoundException; +import java.io.FileOutputStream; +import java.io.FilterInputStream; +import java.io.IOException; +import java.io.InputStream; +import java.nio.BufferUnderflowException; +import java.nio.ByteBuffer; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; + +import com.google.common.base.Preconditions; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.fs.FSExceptionMessages; +import org.apache.hadoop.util.DirectBufferPool; + +import static org.apache.hadoop.fs.s3a.S3ADataBlocks.DataBlock.DestState.*; + +/** + * Set of classes to support output streaming into blocks which are then + * uploaded as partitions. + */ +final class S3ADataBlocks { + + private static final Logger LOG = LoggerFactory.getLogger(S3ADataBlocks.class); + + private S3ADataBlocks() { + } + + /** + * Validate args to a write command. These are the same validation checks + * expected for any implementation of {@code OutputStream.write()} . + * @param b byte array containing data + * @param off offset in array where to start + * @param len number of bytes to be written + * @throws NullPointerException for a null buffer + * @throws IndexOutOfBoundsException if indices are out of range + */ + static void validateWriteArgs(byte[] b, int off, int len) + throws IOException { + Preconditions.checkNotNull(b); + if ((off < 0) || (off > b.length) || (len < 0) || + ((off + len) > b.length) || ((off + len) < 0)) { + throw new IndexOutOfBoundsException( + "write (b[" + b.length + "], " + off + ", " + len + ')'); + } + } + + /** + * Create a factory. + * @param owner factory owner + * @param name factory name -the option from {@link Constants} . + * @return the factory, ready to be initialized. + * @throws IllegalArgumentException if the name is unknown. + */ + static BlockFactory createFactory(S3AFileSystem owner, + String name) { + switch (name) { + case Constants.BLOCK_OUTPUT_BUFFER_ARRAY: + return new ArrayBlockFactory(owner); + case Constants.BLOCK_OUTPUT_BUFFER_DISK: + return new DiskBlockFactory(owner); + case Constants.BLOCK_OUTPUT_BYTEBUFFER: + return new ByteBufferBlockFactory(owner); + default: + throw new IllegalArgumentException("Unsupported block buffer" + + " \"" + name + '"'); + } + } + + /** + * Base class for block factories. + */ + static abstract class BlockFactory implements Closeable { + + /** + * Owner. + */ + protected final S3AFileSystem owner; + + protected BlockFactory(S3AFileSystem owner) { + this.owner = owner; + } + + /** + * Create a block. + * @param limit limit of the block. + * @return a new block. + */ + abstract DataBlock create(int limit) throws IOException; + + /** + * Implement any close/cleanup operation. + * Base class is a no-op + * @throws IOException -ideally, it shouldn't. + */ + @Override + public void close() throws IOException { + } + } + + /** + * This represents a block being uploaded. + */ + static abstract class DataBlock implements Closeable { + + private volatile DestState state = Writing; + + /** + * Enter + * @param current + * @param next + * @throws IllegalStateException + */ + protected synchronized final void enterState(DestState current, + DestState next) + throws IllegalStateException { + verifyState(current); + LOG.debug("{}: entering state {}", this, next); + state = next; + } + + /** + * Verify that the block is in the declared state. + * @param expected expected state. + * @throws IllegalStateException if the DataBlock is in the wrong state + */ + protected final void verifyState(DestState expected) throws IllegalStateException { + if (expected != null && state != expected) { + throw new IllegalStateException("Expected stream state " + expected + + " -but actual state is " + state + " in " + this); + } + } + + /** + * Current state. + * @return the current state. + */ + final DestState getState() { + return state; + } + + /** + * Return the current data size. + * @return the size of the data + */ + abstract int dataSize(); + + /** + * Predicate to verify that the block has the capacity to write + * the given set of bytes. + * @param bytes number of bytes desired to be written. + * @return true if there is enough space. + */ + abstract boolean hasCapacity(long bytes); + + /** + * Predicate to check if there is data in the block. + * @return true if there is + */ + boolean hasData() { + return dataSize() > 0; + } + + /** + * The remaining capacity in the block before it is full. + * @return the number of bytes remaining. + */ + abstract int remainingCapacity(); + + /** + * Write a series of bytes from the buffer, from the offset. + * Returns the number of bytes written. + * Only valid in the state {@code Writing}. + * Base class verifies the state but does no writing. + * @param buffer buffer + * @param offset offset + * @param length length of write + * @return number of bytes written + * @throws IOException trouble + */ + int write(byte[] buffer, int offset, int length) throws IOException { + verifyState(Writing); + Preconditions.checkArgument(buffer != null, "Null buffer"); + Preconditions.checkArgument(length >= 0, "length is negative"); + Preconditions.checkArgument(offset >= 0, "offset is negative"); + Preconditions.checkArgument( + !(buffer.length - offset < length), + "buffer shorter than amount of data to write"); + return 0; + } + + /** + * Flush the output. + * Only valid in the state {@code Writing} . + * In the base class, this is a no-op + * @throws IOException any IO problem. + */ + void flush() throws IOException { + verifyState(Writing); + } + + /** + * Switch to the upload state and return a stream for uploading. + * Base class calls {@link #enterState(DestState, DestState)} to + * manage the state machine. + * @return the stream + * @throws IOException trouble + */ + InputStream startUpload() throws IOException { + LOG.debug("Start datablock upload"); + enterState(Writing, Upload); + return null; + } + + /** + * Enter the closed state. + * @return true if the class was in any other state, implying that + * the subclass should do its close operations + */ + protected synchronized boolean enterClosedState() { + if (!state.equals(Closed)) { + try { + enterState(null, Closed); + } catch (IllegalStateException ignored) { — End diff – I know it can't happen, but like to close off all failure routes of a close() call. Anyway, throwing again.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user steveloughran commented on a diff in the pull request:

          https://github.com/apache/hadoop/pull/130#discussion_r82171318

          — Diff: hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ADataBlocks.java —
          @@ -0,0 +1,816 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +
          +package org.apache.hadoop.fs.s3a;
          +
          +import java.io.BufferedOutputStream;
          +import java.io.ByteArrayInputStream;
          +import java.io.ByteArrayOutputStream;
          +import java.io.Closeable;
          +import java.io.EOFException;
          +import java.io.File;
          +import java.io.FileInputStream;
          +import java.io.FileNotFoundException;
          +import java.io.FileOutputStream;
          +import java.io.FilterInputStream;
          +import java.io.IOException;
          +import java.io.InputStream;
          +import java.nio.BufferUnderflowException;
          +import java.nio.ByteBuffer;
          +import java.util.concurrent.atomic.AtomicBoolean;
          +import java.util.concurrent.atomic.AtomicInteger;
          +
          +import com.google.common.base.Preconditions;
          +import org.slf4j.Logger;
          +import org.slf4j.LoggerFactory;
          +
          +import org.apache.hadoop.fs.FSExceptionMessages;
          +import org.apache.hadoop.util.DirectBufferPool;
          +
          +import static org.apache.hadoop.fs.s3a.S3ADataBlocks.DataBlock.DestState.*;
          +
          +/**
          + * Set of classes to support output streaming into blocks which are then
          + * uploaded as partitions.
          + */
          +final class S3ADataBlocks {
          +
          + private static final Logger LOG = LoggerFactory.getLogger(S3ADataBlocks.class);
          +
          + private S3ADataBlocks()

          { + }

          +
          + /**
          + * Validate args to a write command. These are the same validation checks
          + * expected for any implementation of

          {@code OutputStream.write()}

          .
          + * @param b byte array containing data
          + * @param off offset in array where to start
          + * @param len number of bytes to be written
          + * @throws NullPointerException for a null buffer
          + * @throws IndexOutOfBoundsException if indices are out of range
          + */
          + static void validateWriteArgs(byte[] b, int off, int len)
          + throws IOException {
          + Preconditions.checkNotNull(b);
          + if ((off < 0) || (off > b.length) || (len < 0) ||
          + ((off + len) > b.length) || ((off + len) < 0))

          { + throw new IndexOutOfBoundsException( + "write (b[" + b.length + "], " + off + ", " + len + ')'); + }

          + }
          +
          + /**
          + * Create a factory.
          + * @param owner factory owner
          + * @param name factory name -the option from

          {@link Constants}

          .
          + * @return the factory, ready to be initialized.
          + * @throws IllegalArgumentException if the name is unknown.
          + */
          + static BlockFactory createFactory(S3AFileSystem owner,
          + String name) {
          + switch (name)

          { + case Constants.BLOCK_OUTPUT_BUFFER_ARRAY: + return new ArrayBlockFactory(owner); + case Constants.BLOCK_OUTPUT_BUFFER_DISK: + return new DiskBlockFactory(owner); + case Constants.BLOCK_OUTPUT_BYTEBUFFER: + return new ByteBufferBlockFactory(owner); + default: + throw new IllegalArgumentException("Unsupported block buffer" + + " \"" + name + '"'); + }

          + }
          +
          + /**
          + * Base class for block factories.
          + */
          + static abstract class BlockFactory implements Closeable {
          +
          + /**
          + * Owner.
          + */
          + protected final S3AFileSystem owner;
          +
          + protected BlockFactory(S3AFileSystem owner)

          { + this.owner = owner; + }

          +
          + /**
          + * Create a block.
          + * @param limit limit of the block.
          + * @return a new block.
          + */
          + abstract DataBlock create(int limit) throws IOException;
          +
          + /**
          + * Implement any close/cleanup operation.
          + * Base class is a no-op
          + * @throws IOException -ideally, it shouldn't.
          + */
          + @Override
          + public void close() throws IOException

          { + }

          + }
          +
          + /**
          + * This represents a block being uploaded.
          + */
          + static abstract class DataBlock implements Closeable {
          +
          + private volatile DestState state = Writing;
          +
          + /**
          + * Enter
          + * @param current
          + * @param next
          + * @throws IllegalStateException
          + */
          + protected synchronized final void enterState(DestState current,
          + DestState next)
          + throws IllegalStateException {
          + verifyState(current);
          + LOG.debug("{}: entering state {}", this, next);
          + state = next;
          + }
          +
          + /**
          + * Verify that the block is in the declared state.
          + * @param expected expected state.
          + * @throws IllegalStateException if the DataBlock is in the wrong state
          + */
          + protected final void verifyState(DestState expected) throws IllegalStateException {
          + if (expected != null && state != expected)

          { + throw new IllegalStateException("Expected stream state " + expected + + " -but actual state is " + state + " in " + this); + }

          + }
          +
          + /**
          + * Current state.
          + * @return the current state.
          + */
          + final DestState getState()

          { + return state; + }

          +
          + /**
          + * Return the current data size.
          + * @return the size of the data
          + */
          + abstract int dataSize();
          +
          + /**
          + * Predicate to verify that the block has the capacity to write
          + * the given set of bytes.
          + * @param bytes number of bytes desired to be written.
          + * @return true if there is enough space.
          + */
          + abstract boolean hasCapacity(long bytes);
          +
          + /**
          + * Predicate to check if there is data in the block.
          + * @return true if there is
          + */
          + boolean hasData()

          { + return dataSize() > 0; + }

          +
          + /**
          + * The remaining capacity in the block before it is full.
          + * @return the number of bytes remaining.
          + */
          + abstract int remainingCapacity();
          +
          + /**
          + * Write a series of bytes from the buffer, from the offset.
          + * Returns the number of bytes written.
          + * Only valid in the state

          {@code Writing}.
          + * Base class verifies the state but does no writing.
          + * @param buffer buffer
          + * @param offset offset
          + * @param length length of write
          + * @return number of bytes written
          + * @throws IOException trouble
          + */
          + int write(byte[] buffer, int offset, int length) throws IOException { + verifyState(Writing); + Preconditions.checkArgument(buffer != null, "Null buffer"); + Preconditions.checkArgument(length >= 0, "length is negative"); + Preconditions.checkArgument(offset >= 0, "offset is negative"); + Preconditions.checkArgument( + !(buffer.length - offset < length), + "buffer shorter than amount of data to write"); + return 0; + }
          +
          + /**
          + * Flush the output.
          + * Only valid in the state {@code Writing}

          .
          + * In the base class, this is a no-op
          + * @throws IOException any IO problem.
          + */
          + void flush() throws IOException

          { + verifyState(Writing); + }

          +
          + /**
          + * Switch to the upload state and return a stream for uploading.
          + * Base class calls

          {@link #enterState(DestState, DestState)}

          to
          + * manage the state machine.
          + * @return the stream
          + * @throws IOException trouble
          + */
          + InputStream startUpload() throws IOException

          { + LOG.debug("Start datablock upload"); + enterState(Writing, Upload); + return null; + }

          +
          + /**
          + * Enter the closed state.
          + * @return true if the class was in any other state, implying that
          + * the subclass should do its close operations
          + */
          + protected synchronized boolean enterClosedState() {
          + if (!state.equals(Closed)) {
          + try

          { + enterState(null, Closed); + }

          catch (IllegalStateException ignored)

          { + + }

          + return true;
          + } else

          { + return false; + }

          + }
          +
          + @Override
          + public void close() throws IOException {
          + if (enterClosedState()) {
          + LOG.debug("Closed {}", this);
          + innerClose();
          + }
          + }
          +
          + /**
          + * Inner close logic for subclasses to implement.
          + */
          + protected void innerClose() throws IOException

          { + + }

          +
          + enum DestState

          {Writing, Upload, Closed}

          + }
          +
          + // ====================================================================
          +
          + /**
          + * Use byte arrays on the heap for storage.
          + */
          + static class ArrayBlockFactory extends BlockFactory {
          +
          + ArrayBlockFactory(S3AFileSystem owner)

          { + super(owner); + }

          +
          + @Override
          + DataBlock create(int limit) throws IOException

          { + return new ByteArrayBlock(limit); + }

          +
          + }
          +
          + /**
          + * Stream to memory via a

          {@code ByteArrayOutputStream}

          .
          + *
          + * This was taken from

          {@code S3AFastOutputStream}

          and has the
          + * same problem which surfaced there: it consumes heap space
          + * proportional to the mismatch between writes to the stream and
          + * the JVM-wide upload bandwidth to the S3 endpoint.
          + */
          +
          + static class ByteArrayBlock extends DataBlock {
          + private ByteArrayOutputStream buffer;
          + private final int limit;
          + // cache data size so that it is consistent after the buffer is reset.
          + private Integer dataSize;
          +
          + ByteArrayBlock(int limit)

          { + this.limit = limit; + buffer = new ByteArrayOutputStream(); + }

          +
          + /**
          + * Get the amount of data; if there is no buffer then the size is 0.
          + * @return the amount of data available to upload.
          + */
          + @Override
          + int dataSize()

          { + return dataSize != null ? dataSize : buffer.size(); + }

          +
          + @Override
          + InputStream startUpload() throws IOException {
          + super.startUpload();
          + dataSize = buffer.size();
          + ByteArrayInputStream bufferData = new ByteArrayInputStream(
          + buffer.toByteArray());
          + buffer.reset();
          — End diff –

          OK

          Show
          githubbot ASF GitHub Bot added a comment - Github user steveloughran commented on a diff in the pull request: https://github.com/apache/hadoop/pull/130#discussion_r82171318 — Diff: hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ADataBlocks.java — @@ -0,0 +1,816 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.s3a; + +import java.io.BufferedOutputStream; +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.Closeable; +import java.io.EOFException; +import java.io.File; +import java.io.FileInputStream; +import java.io.FileNotFoundException; +import java.io.FileOutputStream; +import java.io.FilterInputStream; +import java.io.IOException; +import java.io.InputStream; +import java.nio.BufferUnderflowException; +import java.nio.ByteBuffer; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; + +import com.google.common.base.Preconditions; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.fs.FSExceptionMessages; +import org.apache.hadoop.util.DirectBufferPool; + +import static org.apache.hadoop.fs.s3a.S3ADataBlocks.DataBlock.DestState.*; + +/** + * Set of classes to support output streaming into blocks which are then + * uploaded as partitions. + */ +final class S3ADataBlocks { + + private static final Logger LOG = LoggerFactory.getLogger(S3ADataBlocks.class); + + private S3ADataBlocks() { + } + + /** + * Validate args to a write command. These are the same validation checks + * expected for any implementation of {@code OutputStream.write()} . + * @param b byte array containing data + * @param off offset in array where to start + * @param len number of bytes to be written + * @throws NullPointerException for a null buffer + * @throws IndexOutOfBoundsException if indices are out of range + */ + static void validateWriteArgs(byte[] b, int off, int len) + throws IOException { + Preconditions.checkNotNull(b); + if ((off < 0) || (off > b.length) || (len < 0) || + ((off + len) > b.length) || ((off + len) < 0)) { + throw new IndexOutOfBoundsException( + "write (b[" + b.length + "], " + off + ", " + len + ')'); + } + } + + /** + * Create a factory. + * @param owner factory owner + * @param name factory name -the option from {@link Constants} . + * @return the factory, ready to be initialized. + * @throws IllegalArgumentException if the name is unknown. + */ + static BlockFactory createFactory(S3AFileSystem owner, + String name) { + switch (name) { + case Constants.BLOCK_OUTPUT_BUFFER_ARRAY: + return new ArrayBlockFactory(owner); + case Constants.BLOCK_OUTPUT_BUFFER_DISK: + return new DiskBlockFactory(owner); + case Constants.BLOCK_OUTPUT_BYTEBUFFER: + return new ByteBufferBlockFactory(owner); + default: + throw new IllegalArgumentException("Unsupported block buffer" + + " \"" + name + '"'); + } + } + + /** + * Base class for block factories. + */ + static abstract class BlockFactory implements Closeable { + + /** + * Owner. + */ + protected final S3AFileSystem owner; + + protected BlockFactory(S3AFileSystem owner) { + this.owner = owner; + } + + /** + * Create a block. + * @param limit limit of the block. + * @return a new block. + */ + abstract DataBlock create(int limit) throws IOException; + + /** + * Implement any close/cleanup operation. + * Base class is a no-op + * @throws IOException -ideally, it shouldn't. + */ + @Override + public void close() throws IOException { + } + } + + /** + * This represents a block being uploaded. + */ + static abstract class DataBlock implements Closeable { + + private volatile DestState state = Writing; + + /** + * Enter + * @param current + * @param next + * @throws IllegalStateException + */ + protected synchronized final void enterState(DestState current, + DestState next) + throws IllegalStateException { + verifyState(current); + LOG.debug("{}: entering state {}", this, next); + state = next; + } + + /** + * Verify that the block is in the declared state. + * @param expected expected state. + * @throws IllegalStateException if the DataBlock is in the wrong state + */ + protected final void verifyState(DestState expected) throws IllegalStateException { + if (expected != null && state != expected) { + throw new IllegalStateException("Expected stream state " + expected + + " -but actual state is " + state + " in " + this); + } + } + + /** + * Current state. + * @return the current state. + */ + final DestState getState() { + return state; + } + + /** + * Return the current data size. + * @return the size of the data + */ + abstract int dataSize(); + + /** + * Predicate to verify that the block has the capacity to write + * the given set of bytes. + * @param bytes number of bytes desired to be written. + * @return true if there is enough space. + */ + abstract boolean hasCapacity(long bytes); + + /** + * Predicate to check if there is data in the block. + * @return true if there is + */ + boolean hasData() { + return dataSize() > 0; + } + + /** + * The remaining capacity in the block before it is full. + * @return the number of bytes remaining. + */ + abstract int remainingCapacity(); + + /** + * Write a series of bytes from the buffer, from the offset. + * Returns the number of bytes written. + * Only valid in the state {@code Writing}. + * Base class verifies the state but does no writing. + * @param buffer buffer + * @param offset offset + * @param length length of write + * @return number of bytes written + * @throws IOException trouble + */ + int write(byte[] buffer, int offset, int length) throws IOException { + verifyState(Writing); + Preconditions.checkArgument(buffer != null, "Null buffer"); + Preconditions.checkArgument(length >= 0, "length is negative"); + Preconditions.checkArgument(offset >= 0, "offset is negative"); + Preconditions.checkArgument( + !(buffer.length - offset < length), + "buffer shorter than amount of data to write"); + return 0; + } + + /** + * Flush the output. + * Only valid in the state {@code Writing} . + * In the base class, this is a no-op + * @throws IOException any IO problem. + */ + void flush() throws IOException { + verifyState(Writing); + } + + /** + * Switch to the upload state and return a stream for uploading. + * Base class calls {@link #enterState(DestState, DestState)} to + * manage the state machine. + * @return the stream + * @throws IOException trouble + */ + InputStream startUpload() throws IOException { + LOG.debug("Start datablock upload"); + enterState(Writing, Upload); + return null; + } + + /** + * Enter the closed state. + * @return true if the class was in any other state, implying that + * the subclass should do its close operations + */ + protected synchronized boolean enterClosedState() { + if (!state.equals(Closed)) { + try { + enterState(null, Closed); + } catch (IllegalStateException ignored) { + + } + return true; + } else { + return false; + } + } + + @Override + public void close() throws IOException { + if (enterClosedState()) { + LOG.debug("Closed {}", this); + innerClose(); + } + } + + /** + * Inner close logic for subclasses to implement. + */ + protected void innerClose() throws IOException { + + } + + enum DestState {Writing, Upload, Closed} + } + + // ==================================================================== + + /** + * Use byte arrays on the heap for storage. + */ + static class ArrayBlockFactory extends BlockFactory { + + ArrayBlockFactory(S3AFileSystem owner) { + super(owner); + } + + @Override + DataBlock create(int limit) throws IOException { + return new ByteArrayBlock(limit); + } + + } + + /** + * Stream to memory via a {@code ByteArrayOutputStream} . + * + * This was taken from {@code S3AFastOutputStream} and has the + * same problem which surfaced there: it consumes heap space + * proportional to the mismatch between writes to the stream and + * the JVM-wide upload bandwidth to the S3 endpoint. + */ + + static class ByteArrayBlock extends DataBlock { + private ByteArrayOutputStream buffer; + private final int limit; + // cache data size so that it is consistent after the buffer is reset. + private Integer dataSize; + + ByteArrayBlock(int limit) { + this.limit = limit; + buffer = new ByteArrayOutputStream(); + } + + /** + * Get the amount of data; if there is no buffer then the size is 0. + * @return the amount of data available to upload. + */ + @Override + int dataSize() { + return dataSize != null ? dataSize : buffer.size(); + } + + @Override + InputStream startUpload() throws IOException { + super.startUpload(); + dataSize = buffer.size(); + ByteArrayInputStream bufferData = new ByteArrayInputStream( + buffer.toByteArray()); + buffer.reset(); — End diff – OK
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user steveloughran commented on a diff in the pull request:

          https://github.com/apache/hadoop/pull/130#discussion_r82172654

          — Diff: hadoop-common-project/hadoop-common/src/main/resources/core-default.xml —
          @@ -1093,12 +1101,48 @@
          </property>

          <property>

          • <name>fs.s3a.fast.upload</name>
            + <name>fs.s3a.block.output</name>
              • End diff –

          afraid so, that bit had missed the push as I forgot to --force the patch up at the end of the day. Will push it up with all the comments here, after another test run

          Show
          githubbot ASF GitHub Bot added a comment - Github user steveloughran commented on a diff in the pull request: https://github.com/apache/hadoop/pull/130#discussion_r82172654 — Diff: hadoop-common-project/hadoop-common/src/main/resources/core-default.xml — @@ -1093,12 +1101,48 @@ </property> <property> <name>fs.s3a.fast.upload</name> + <name>fs.s3a.block.output</name> End diff – afraid so, that bit had missed the push as I forgot to --force the patch up at the end of the day. Will push it up with all the comments here, after another test run
          Hide
          stevel@apache.org Steve Loughran added a comment -

          Yetus isn't picking this up. as well as resubmitting, i'm going to create another JIRA with the patches coming in as .patch files, rather than a PR

          Show
          stevel@apache.org Steve Loughran added a comment - Yetus isn't picking this up. as well as resubmitting, i'm going to create another JIRA with the patches coming in as .patch files, rather than a PR
          Hide
          stevel@apache.org Steve Loughran added a comment - - edited

          Moving patch testing to HADOOP-13703

          Show
          stevel@apache.org Steve Loughran added a comment - - edited Moving patch testing to HADOOP-13703
          Hide
          cnauroth Chris Nauroth added a comment -

          Steve Loughran, I reviewed HADOOP-13560-branch-2-011.patch attached to HADOOP-13703. This version looks good to me. My only remaining request is to remove an unused import in ITestS3AHugeFilesByteBufferBlocks, and that's trivial enough to fix on commit. We also need another patch file for trunk. After that, I'll be ready to sign off.

          Show
          cnauroth Chris Nauroth added a comment - Steve Loughran , I reviewed HADOOP-13560 -branch-2-011.patch attached to HADOOP-13703 . This version looks good to me. My only remaining request is to remove an unused import in ITestS3AHugeFilesByteBufferBlocks , and that's trivial enough to fix on commit. We also need another patch file for trunk. After that, I'll be ready to sign off.
          Hide
          stevel@apache.org Steve Loughran added a comment -

          that unused import is needed; its one of those cases where findbugs is wrong. I'm referring to the import in the javadocs; remove it and the cross reference doesn't apply. So: needed.

          I'll do the trunk branch though.

          Show
          stevel@apache.org Steve Loughran added a comment - that unused import is needed; its one of those cases where findbugs is wrong. I'm referring to the import in the javadocs; remove it and the cross reference doesn't apply. So: needed. I'll do the trunk branch though.
          Hide
          cnauroth Chris Nauroth added a comment -

          I see now. I figured the JavaDoc could just link directly to the constant brought in via import static, but it appears JavaDoc doesn't support that. Please disregard my comment, and I'll look at the trunk patch.

          Show
          cnauroth Chris Nauroth added a comment - I see now. I figured the JavaDoc could just link directly to the constant brought in via import static , but it appears JavaDoc doesn't support that. Please disregard my comment, and I'll look at the trunk patch.
          Hide
          cnauroth Chris Nauroth added a comment -

          The trunk patch looks good to me. I have tested successfully against us-west-2.

          However, I just noticed that after I apply the branch-2 patch and build the distro, every hadoop fs command accessing S3A prints a deprecation warning:

          16/10/12 13:48:53 INFO Configuration.deprecation: Unsupported option "fs.s3a.threads.core" will be ignored
          

          I think that's because the patch added the DeprecationDelta, but S3AFileSystem#initialize still has code that tries to access fs.s3a.threads.core.

          Show
          cnauroth Chris Nauroth added a comment - The trunk patch looks good to me. I have tested successfully against us-west-2. However, I just noticed that after I apply the branch-2 patch and build the distro, every hadoop fs command accessing S3A prints a deprecation warning: 16/10/12 13:48:53 INFO Configuration.deprecation: Unsupported option "fs.s3a.threads.core" will be ignored I think that's because the patch added the DeprecationDelta , but S3AFileSystem#initialize still has code that tries to access fs.s3a.threads.core .
          Hide
          stevel@apache.org Steve Loughran added a comment -

          will fix

          Show
          stevel@apache.org Steve Loughran added a comment - will fix
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user pieterreuse commented on a diff in the pull request:

          https://github.com/apache/hadoop/pull/130#discussion_r83198713

          — Diff: hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ADataBlocks.java —
          @@ -0,0 +1,811 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +
          +package org.apache.hadoop.fs.s3a;
          +
          +import java.io.BufferedOutputStream;
          +import java.io.ByteArrayInputStream;
          +import java.io.ByteArrayOutputStream;
          +import java.io.Closeable;
          +import java.io.EOFException;
          +import java.io.File;
          +import java.io.FileInputStream;
          +import java.io.FileNotFoundException;
          +import java.io.FileOutputStream;
          +import java.io.FilterInputStream;
          +import java.io.IOException;
          +import java.io.InputStream;
          +import java.nio.ByteBuffer;
          +import java.util.concurrent.atomic.AtomicBoolean;
          +import java.util.concurrent.atomic.AtomicInteger;
          +
          +import com.google.common.base.Preconditions;
          +import org.slf4j.Logger;
          +import org.slf4j.LoggerFactory;
          +
          +import org.apache.hadoop.fs.FSExceptionMessages;
          +import org.apache.hadoop.util.DirectBufferPool;
          +
          +import static org.apache.hadoop.fs.s3a.S3ADataBlocks.DataBlock.DestState.*;
          +
          +/**
          + * Set of classes to support output streaming into blocks which are then
          + * uploaded as partitions.
          + */
          +final class S3ADataBlocks {
          +
          + private static final Logger LOG = LoggerFactory.getLogger(S3ADataBlocks.class);
          +
          + private S3ADataBlocks()

          { + }

          +
          + /**
          + * Validate args to a write command. These are the same validation checks
          + * expected for any implementation of

          {@code OutputStream.write()}

          .
          + * @param b byte array containing data
          + * @param off offset in array where to start
          + * @param len number of bytes to be written
          + * @throws NullPointerException for a null buffer
          + * @throws IndexOutOfBoundsException if indices are out of range
          + */
          + static void validateWriteArgs(byte[] b, int off, int len)
          + throws IOException {
          + Preconditions.checkNotNull(b);
          + if ((off < 0) || (off > b.length) || (len < 0) ||
          + ((off + len) > b.length) || ((off + len) < 0))

          { + throw new IndexOutOfBoundsException( + "write (b[" + b.length + "], " + off + ", " + len + ')'); + }

          + }
          +
          + /**
          + * Create a factory.
          + * @param owner factory owner
          + * @param name factory name -the option from

          {@link Constants}

          .
          + * @return the factory, ready to be initialized.
          + * @throws IllegalArgumentException if the name is unknown.
          + */
          + static BlockFactory createFactory(S3AFileSystem owner,
          + String name) {
          + switch (name)

          { + case Constants.FAST_UPLOAD_BUFFER_ARRAY: + return new ArrayBlockFactory(owner); + case Constants.FAST_UPLOAD_BUFFER_DISK: + return new DiskBlockFactory(owner); + case Constants.FAST_UPLOAD_BYTEBUFFER: + return new ByteBufferBlockFactory(owner); + default: + throw new IllegalArgumentException("Unsupported block buffer" + + " \"" + name + '"'); + }

          + }
          +
          + /**
          + * Base class for block factories.
          + */
          + static abstract class BlockFactory implements Closeable {
          +
          + /**
          + * Owner.
          + */
          + protected final S3AFileSystem owner;
          +
          + protected BlockFactory(S3AFileSystem owner)

          { + this.owner = owner; + }

          +
          + /**
          + * Create a block.
          + * @param limit limit of the block.
          + * @return a new block.
          + */
          + abstract DataBlock create(int limit) throws IOException;
          +
          + /**
          + * Implement any close/cleanup operation.
          + * Base class is a no-op
          + * @throws IOException -ideally, it shouldn't.
          + */
          + @Override
          + public void close() throws IOException

          { + }

          + }
          +
          + /**
          + * This represents a block being uploaded.
          + */
          + static abstract class DataBlock implements Closeable {
          +
          + private volatile DestState state = Writing;
          — End diff –

          I'm nitpicking here, but wouldn't it make more sense to define DestState here instead of on line 272? Moving that line here would improve code readability imo but wouldn't change any behaviour.

          Show
          githubbot ASF GitHub Bot added a comment - Github user pieterreuse commented on a diff in the pull request: https://github.com/apache/hadoop/pull/130#discussion_r83198713 — Diff: hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ADataBlocks.java — @@ -0,0 +1,811 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.s3a; + +import java.io.BufferedOutputStream; +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.Closeable; +import java.io.EOFException; +import java.io.File; +import java.io.FileInputStream; +import java.io.FileNotFoundException; +import java.io.FileOutputStream; +import java.io.FilterInputStream; +import java.io.IOException; +import java.io.InputStream; +import java.nio.ByteBuffer; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; + +import com.google.common.base.Preconditions; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.fs.FSExceptionMessages; +import org.apache.hadoop.util.DirectBufferPool; + +import static org.apache.hadoop.fs.s3a.S3ADataBlocks.DataBlock.DestState.*; + +/** + * Set of classes to support output streaming into blocks which are then + * uploaded as partitions. + */ +final class S3ADataBlocks { + + private static final Logger LOG = LoggerFactory.getLogger(S3ADataBlocks.class); + + private S3ADataBlocks() { + } + + /** + * Validate args to a write command. These are the same validation checks + * expected for any implementation of {@code OutputStream.write()} . + * @param b byte array containing data + * @param off offset in array where to start + * @param len number of bytes to be written + * @throws NullPointerException for a null buffer + * @throws IndexOutOfBoundsException if indices are out of range + */ + static void validateWriteArgs(byte[] b, int off, int len) + throws IOException { + Preconditions.checkNotNull(b); + if ((off < 0) || (off > b.length) || (len < 0) || + ((off + len) > b.length) || ((off + len) < 0)) { + throw new IndexOutOfBoundsException( + "write (b[" + b.length + "], " + off + ", " + len + ')'); + } + } + + /** + * Create a factory. + * @param owner factory owner + * @param name factory name -the option from {@link Constants} . + * @return the factory, ready to be initialized. + * @throws IllegalArgumentException if the name is unknown. + */ + static BlockFactory createFactory(S3AFileSystem owner, + String name) { + switch (name) { + case Constants.FAST_UPLOAD_BUFFER_ARRAY: + return new ArrayBlockFactory(owner); + case Constants.FAST_UPLOAD_BUFFER_DISK: + return new DiskBlockFactory(owner); + case Constants.FAST_UPLOAD_BYTEBUFFER: + return new ByteBufferBlockFactory(owner); + default: + throw new IllegalArgumentException("Unsupported block buffer" + + " \"" + name + '"'); + } + } + + /** + * Base class for block factories. + */ + static abstract class BlockFactory implements Closeable { + + /** + * Owner. + */ + protected final S3AFileSystem owner; + + protected BlockFactory(S3AFileSystem owner) { + this.owner = owner; + } + + /** + * Create a block. + * @param limit limit of the block. + * @return a new block. + */ + abstract DataBlock create(int limit) throws IOException; + + /** + * Implement any close/cleanup operation. + * Base class is a no-op + * @throws IOException -ideally, it shouldn't. + */ + @Override + public void close() throws IOException { + } + } + + /** + * This represents a block being uploaded. + */ + static abstract class DataBlock implements Closeable { + + private volatile DestState state = Writing; — End diff – I'm nitpicking here, but wouldn't it make more sense to define DestState here instead of on line 272? Moving that line here would improve code readability imo but wouldn't change any behaviour.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user pieterreuse commented on a diff in the pull request:

          https://github.com/apache/hadoop/pull/130#discussion_r83199198

          — Diff: hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java —
          @@ -0,0 +1,699 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +
          +package org.apache.hadoop.fs.s3a;
          +
          +import java.io.IOException;
          +import java.io.InputStream;
          +import java.io.OutputStream;
          +import java.util.ArrayList;
          +import java.util.List;
          +import java.util.concurrent.Callable;
          +import java.util.concurrent.ExecutionException;
          +import java.util.concurrent.ExecutorService;
          +import java.util.concurrent.TimeUnit;
          +import java.util.concurrent.atomic.AtomicBoolean;
          +
          +import com.amazonaws.AmazonClientException;
          +import com.amazonaws.event.ProgressEvent;
          +import com.amazonaws.event.ProgressEventType;
          +import com.amazonaws.event.ProgressListener;
          +import com.amazonaws.services.s3.model.CompleteMultipartUploadResult;
          +import com.amazonaws.services.s3.model.PartETag;
          +import com.amazonaws.services.s3.model.PutObjectRequest;
          +import com.amazonaws.services.s3.model.PutObjectResult;
          +import com.amazonaws.services.s3.model.UploadPartRequest;
          +import com.google.common.base.Preconditions;
          +import com.google.common.util.concurrent.Futures;
          +import com.google.common.util.concurrent.ListenableFuture;
          +import com.google.common.util.concurrent.ListeningExecutorService;
          +import com.google.common.util.concurrent.MoreExecutors;
          +import org.slf4j.Logger;
          +import org.slf4j.LoggerFactory;
          +
          +import org.apache.hadoop.classification.InterfaceAudience;
          +import org.apache.hadoop.classification.InterfaceStability;
          +import org.apache.hadoop.io.IOUtils;
          +import org.apache.hadoop.io.retry.RetryPolicies;
          +import org.apache.hadoop.io.retry.RetryPolicy;
          +import org.apache.hadoop.util.Progressable;
          +
          +import static org.apache.hadoop.fs.s3a.S3AUtils.*;
          +import static org.apache.hadoop.fs.s3a.Statistic.*;
          +
          +/**
          + * Upload files/parts directly via different buffering mechanisms:
          + * including memory and disk.
          + *
          + * If the stream is closed and no update has started, then the upload
          + * is instead done as a single PUT operation.
          + *
          + * Unstable: statistics and error handling might evolve.
          + */
          +@InterfaceAudience.Private
          +@InterfaceStability.Unstable
          +class S3ABlockOutputStream extends OutputStream {
          +
          + private static final Logger LOG =
          + LoggerFactory.getLogger(S3ABlockOutputStream.class);
          +
          + /** Owner FileSystem. */
          + private final S3AFileSystem fs;
          +
          + /** Object being uploaded. */
          + private final String key;
          +
          + /** Size of all blocks. */
          + private final int blockSize;
          +
          + /** Callback for progress. */
          + private final ProgressListener progressListener;
          + private final ListeningExecutorService executorService;
          +
          + /**
          + * Retry policy for multipart commits; not all AWS SDK versions retry that.
          + */
          + private final RetryPolicy retryPolicy =
          + RetryPolicies.retryUpToMaximumCountWithProportionalSleep(
          + 5,
          + 2000,
          + TimeUnit.MILLISECONDS);
          + /**
          + * Factory for blocks.
          + */
          + private final S3ADataBlocks.BlockFactory blockFactory;
          +
          + /** Preallocated byte buffer for writing single characters. */
          + private final byte[] singleCharWrite = new byte[1];
          +
          + /** Multipart upload details; null means none started. */
          + private MultiPartUpload multiPartUpload;
          +
          + /** Closed flag. */
          + private final AtomicBoolean closed = new AtomicBoolean(false);
          +
          + /** Current data block. Null means none currently active */
          + private S3ADataBlocks.DataBlock activeBlock;
          +
          + /** Count of blocks uploaded. */
          + private long blockCount = 0;
          +
          + /** Statistics to build up. */
          + private final S3AInstrumentation.OutputStreamStatistics statistics;
          +
          + /**
          + * Write operation helper; encapsulation of the filesystem operations.
          + */
          + private final S3AFileSystem.WriteOperationHelper writeOperationHelper;
          +
          + /**
          + * An S3A output stream which uploads partitions in a separate pool of
          + * threads; different

          {@link S3ADataBlocks.BlockFactory}

          + * instances can control where data is buffered.
          + *
          + * @param fs S3AFilesystem
          + * @param key S3 object to work on.
          + * @param executorService the executor service to use to schedule work
          + * @param progress report progress in order to prevent timeouts. If
          + * this class implements

          {@code ProgressListener}

          then it will be
          — End diff –

          This method is passed an object, not a class. You probably meant "If this object implements ..."

          Show
          githubbot ASF GitHub Bot added a comment - Github user pieterreuse commented on a diff in the pull request: https://github.com/apache/hadoop/pull/130#discussion_r83199198 — Diff: hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java — @@ -0,0 +1,699 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.s3a; + +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.Callable; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; + +import com.amazonaws.AmazonClientException; +import com.amazonaws.event.ProgressEvent; +import com.amazonaws.event.ProgressEventType; +import com.amazonaws.event.ProgressListener; +import com.amazonaws.services.s3.model.CompleteMultipartUploadResult; +import com.amazonaws.services.s3.model.PartETag; +import com.amazonaws.services.s3.model.PutObjectRequest; +import com.amazonaws.services.s3.model.PutObjectResult; +import com.amazonaws.services.s3.model.UploadPartRequest; +import com.google.common.base.Preconditions; +import com.google.common.util.concurrent.Futures; +import com.google.common.util.concurrent.ListenableFuture; +import com.google.common.util.concurrent.ListeningExecutorService; +import com.google.common.util.concurrent.MoreExecutors; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.io.IOUtils; +import org.apache.hadoop.io.retry.RetryPolicies; +import org.apache.hadoop.io.retry.RetryPolicy; +import org.apache.hadoop.util.Progressable; + +import static org.apache.hadoop.fs.s3a.S3AUtils.*; +import static org.apache.hadoop.fs.s3a.Statistic.*; + +/** + * Upload files/parts directly via different buffering mechanisms: + * including memory and disk. + * + * If the stream is closed and no update has started, then the upload + * is instead done as a single PUT operation. + * + * Unstable: statistics and error handling might evolve. + */ +@InterfaceAudience.Private +@InterfaceStability.Unstable +class S3ABlockOutputStream extends OutputStream { + + private static final Logger LOG = + LoggerFactory.getLogger(S3ABlockOutputStream.class); + + /** Owner FileSystem. */ + private final S3AFileSystem fs; + + /** Object being uploaded. */ + private final String key; + + /** Size of all blocks. */ + private final int blockSize; + + /** Callback for progress. */ + private final ProgressListener progressListener; + private final ListeningExecutorService executorService; + + /** + * Retry policy for multipart commits; not all AWS SDK versions retry that. + */ + private final RetryPolicy retryPolicy = + RetryPolicies.retryUpToMaximumCountWithProportionalSleep( + 5, + 2000, + TimeUnit.MILLISECONDS); + /** + * Factory for blocks. + */ + private final S3ADataBlocks.BlockFactory blockFactory; + + /** Preallocated byte buffer for writing single characters. */ + private final byte[] singleCharWrite = new byte [1] ; + + /** Multipart upload details; null means none started. */ + private MultiPartUpload multiPartUpload; + + /** Closed flag. */ + private final AtomicBoolean closed = new AtomicBoolean(false); + + /** Current data block. Null means none currently active */ + private S3ADataBlocks.DataBlock activeBlock; + + /** Count of blocks uploaded. */ + private long blockCount = 0; + + /** Statistics to build up. */ + private final S3AInstrumentation.OutputStreamStatistics statistics; + + /** + * Write operation helper; encapsulation of the filesystem operations. + */ + private final S3AFileSystem.WriteOperationHelper writeOperationHelper; + + /** + * An S3A output stream which uploads partitions in a separate pool of + * threads; different {@link S3ADataBlocks.BlockFactory} + * instances can control where data is buffered. + * + * @param fs S3AFilesystem + * @param key S3 object to work on. + * @param executorService the executor service to use to schedule work + * @param progress report progress in order to prevent timeouts. If + * this class implements {@code ProgressListener} then it will be — End diff – This method is passed an object, not a class. You probably meant "If this object implements ..."
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user steveloughran commented on a diff in the pull request:

          https://github.com/apache/hadoop/pull/130#discussion_r83244015

          — Diff: hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ADataBlocks.java —
          @@ -0,0 +1,811 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +
          +package org.apache.hadoop.fs.s3a;
          +
          +import java.io.BufferedOutputStream;
          +import java.io.ByteArrayInputStream;
          +import java.io.ByteArrayOutputStream;
          +import java.io.Closeable;
          +import java.io.EOFException;
          +import java.io.File;
          +import java.io.FileInputStream;
          +import java.io.FileNotFoundException;
          +import java.io.FileOutputStream;
          +import java.io.FilterInputStream;
          +import java.io.IOException;
          +import java.io.InputStream;
          +import java.nio.ByteBuffer;
          +import java.util.concurrent.atomic.AtomicBoolean;
          +import java.util.concurrent.atomic.AtomicInteger;
          +
          +import com.google.common.base.Preconditions;
          +import org.slf4j.Logger;
          +import org.slf4j.LoggerFactory;
          +
          +import org.apache.hadoop.fs.FSExceptionMessages;
          +import org.apache.hadoop.util.DirectBufferPool;
          +
          +import static org.apache.hadoop.fs.s3a.S3ADataBlocks.DataBlock.DestState.*;
          +
          +/**
          + * Set of classes to support output streaming into blocks which are then
          + * uploaded as partitions.
          + */
          +final class S3ADataBlocks {
          +
          + private static final Logger LOG = LoggerFactory.getLogger(S3ADataBlocks.class);
          +
          + private S3ADataBlocks()

          { + }

          +
          + /**
          + * Validate args to a write command. These are the same validation checks
          + * expected for any implementation of

          {@code OutputStream.write()}

          .
          + * @param b byte array containing data
          + * @param off offset in array where to start
          + * @param len number of bytes to be written
          + * @throws NullPointerException for a null buffer
          + * @throws IndexOutOfBoundsException if indices are out of range
          + */
          + static void validateWriteArgs(byte[] b, int off, int len)
          + throws IOException {
          + Preconditions.checkNotNull(b);
          + if ((off < 0) || (off > b.length) || (len < 0) ||
          + ((off + len) > b.length) || ((off + len) < 0))

          { + throw new IndexOutOfBoundsException( + "write (b[" + b.length + "], " + off + ", " + len + ')'); + }

          + }
          +
          + /**
          + * Create a factory.
          + * @param owner factory owner
          + * @param name factory name -the option from

          {@link Constants}

          .
          + * @return the factory, ready to be initialized.
          + * @throws IllegalArgumentException if the name is unknown.
          + */
          + static BlockFactory createFactory(S3AFileSystem owner,
          + String name) {
          + switch (name)

          { + case Constants.FAST_UPLOAD_BUFFER_ARRAY: + return new ArrayBlockFactory(owner); + case Constants.FAST_UPLOAD_BUFFER_DISK: + return new DiskBlockFactory(owner); + case Constants.FAST_UPLOAD_BYTEBUFFER: + return new ByteBufferBlockFactory(owner); + default: + throw new IllegalArgumentException("Unsupported block buffer" + + " \"" + name + '"'); + }

          + }
          +
          + /**
          + * Base class for block factories.
          + */
          + static abstract class BlockFactory implements Closeable {
          +
          + /**
          + * Owner.
          + */
          + protected final S3AFileSystem owner;
          +
          + protected BlockFactory(S3AFileSystem owner)

          { + this.owner = owner; + }

          +
          + /**
          + * Create a block.
          + * @param limit limit of the block.
          + * @return a new block.
          + */
          + abstract DataBlock create(int limit) throws IOException;
          +
          + /**
          + * Implement any close/cleanup operation.
          + * Base class is a no-op
          + * @throws IOException -ideally, it shouldn't.
          + */
          + @Override
          + public void close() throws IOException

          { + }

          + }
          +
          + /**
          + * This represents a block being uploaded.
          + */
          + static abstract class DataBlock implements Closeable {
          +
          + private volatile DestState state = Writing;
          — End diff –

          done

          Show
          githubbot ASF GitHub Bot added a comment - Github user steveloughran commented on a diff in the pull request: https://github.com/apache/hadoop/pull/130#discussion_r83244015 — Diff: hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ADataBlocks.java — @@ -0,0 +1,811 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.s3a; + +import java.io.BufferedOutputStream; +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.Closeable; +import java.io.EOFException; +import java.io.File; +import java.io.FileInputStream; +import java.io.FileNotFoundException; +import java.io.FileOutputStream; +import java.io.FilterInputStream; +import java.io.IOException; +import java.io.InputStream; +import java.nio.ByteBuffer; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; + +import com.google.common.base.Preconditions; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.fs.FSExceptionMessages; +import org.apache.hadoop.util.DirectBufferPool; + +import static org.apache.hadoop.fs.s3a.S3ADataBlocks.DataBlock.DestState.*; + +/** + * Set of classes to support output streaming into blocks which are then + * uploaded as partitions. + */ +final class S3ADataBlocks { + + private static final Logger LOG = LoggerFactory.getLogger(S3ADataBlocks.class); + + private S3ADataBlocks() { + } + + /** + * Validate args to a write command. These are the same validation checks + * expected for any implementation of {@code OutputStream.write()} . + * @param b byte array containing data + * @param off offset in array where to start + * @param len number of bytes to be written + * @throws NullPointerException for a null buffer + * @throws IndexOutOfBoundsException if indices are out of range + */ + static void validateWriteArgs(byte[] b, int off, int len) + throws IOException { + Preconditions.checkNotNull(b); + if ((off < 0) || (off > b.length) || (len < 0) || + ((off + len) > b.length) || ((off + len) < 0)) { + throw new IndexOutOfBoundsException( + "write (b[" + b.length + "], " + off + ", " + len + ')'); + } + } + + /** + * Create a factory. + * @param owner factory owner + * @param name factory name -the option from {@link Constants} . + * @return the factory, ready to be initialized. + * @throws IllegalArgumentException if the name is unknown. + */ + static BlockFactory createFactory(S3AFileSystem owner, + String name) { + switch (name) { + case Constants.FAST_UPLOAD_BUFFER_ARRAY: + return new ArrayBlockFactory(owner); + case Constants.FAST_UPLOAD_BUFFER_DISK: + return new DiskBlockFactory(owner); + case Constants.FAST_UPLOAD_BYTEBUFFER: + return new ByteBufferBlockFactory(owner); + default: + throw new IllegalArgumentException("Unsupported block buffer" + + " \"" + name + '"'); + } + } + + /** + * Base class for block factories. + */ + static abstract class BlockFactory implements Closeable { + + /** + * Owner. + */ + protected final S3AFileSystem owner; + + protected BlockFactory(S3AFileSystem owner) { + this.owner = owner; + } + + /** + * Create a block. + * @param limit limit of the block. + * @return a new block. + */ + abstract DataBlock create(int limit) throws IOException; + + /** + * Implement any close/cleanup operation. + * Base class is a no-op + * @throws IOException -ideally, it shouldn't. + */ + @Override + public void close() throws IOException { + } + } + + /** + * This represents a block being uploaded. + */ + static abstract class DataBlock implements Closeable { + + private volatile DestState state = Writing; — End diff – done
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user steveloughran commented on a diff in the pull request:

          https://github.com/apache/hadoop/pull/130#discussion_r83244207

          — Diff: hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java —
          @@ -0,0 +1,699 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +
          +package org.apache.hadoop.fs.s3a;
          +
          +import java.io.IOException;
          +import java.io.InputStream;
          +import java.io.OutputStream;
          +import java.util.ArrayList;
          +import java.util.List;
          +import java.util.concurrent.Callable;
          +import java.util.concurrent.ExecutionException;
          +import java.util.concurrent.ExecutorService;
          +import java.util.concurrent.TimeUnit;
          +import java.util.concurrent.atomic.AtomicBoolean;
          +
          +import com.amazonaws.AmazonClientException;
          +import com.amazonaws.event.ProgressEvent;
          +import com.amazonaws.event.ProgressEventType;
          +import com.amazonaws.event.ProgressListener;
          +import com.amazonaws.services.s3.model.CompleteMultipartUploadResult;
          +import com.amazonaws.services.s3.model.PartETag;
          +import com.amazonaws.services.s3.model.PutObjectRequest;
          +import com.amazonaws.services.s3.model.PutObjectResult;
          +import com.amazonaws.services.s3.model.UploadPartRequest;
          +import com.google.common.base.Preconditions;
          +import com.google.common.util.concurrent.Futures;
          +import com.google.common.util.concurrent.ListenableFuture;
          +import com.google.common.util.concurrent.ListeningExecutorService;
          +import com.google.common.util.concurrent.MoreExecutors;
          +import org.slf4j.Logger;
          +import org.slf4j.LoggerFactory;
          +
          +import org.apache.hadoop.classification.InterfaceAudience;
          +import org.apache.hadoop.classification.InterfaceStability;
          +import org.apache.hadoop.io.IOUtils;
          +import org.apache.hadoop.io.retry.RetryPolicies;
          +import org.apache.hadoop.io.retry.RetryPolicy;
          +import org.apache.hadoop.util.Progressable;
          +
          +import static org.apache.hadoop.fs.s3a.S3AUtils.*;
          +import static org.apache.hadoop.fs.s3a.Statistic.*;
          +
          +/**
          + * Upload files/parts directly via different buffering mechanisms:
          + * including memory and disk.
          + *
          + * If the stream is closed and no update has started, then the upload
          + * is instead done as a single PUT operation.
          + *
          + * Unstable: statistics and error handling might evolve.
          + */
          +@InterfaceAudience.Private
          +@InterfaceStability.Unstable
          +class S3ABlockOutputStream extends OutputStream {
          +
          + private static final Logger LOG =
          + LoggerFactory.getLogger(S3ABlockOutputStream.class);
          +
          + /** Owner FileSystem. */
          + private final S3AFileSystem fs;
          +
          + /** Object being uploaded. */
          + private final String key;
          +
          + /** Size of all blocks. */
          + private final int blockSize;
          +
          + /** Callback for progress. */
          + private final ProgressListener progressListener;
          + private final ListeningExecutorService executorService;
          +
          + /**
          + * Retry policy for multipart commits; not all AWS SDK versions retry that.
          + */
          + private final RetryPolicy retryPolicy =
          + RetryPolicies.retryUpToMaximumCountWithProportionalSleep(
          + 5,
          + 2000,
          + TimeUnit.MILLISECONDS);
          + /**
          + * Factory for blocks.
          + */
          + private final S3ADataBlocks.BlockFactory blockFactory;
          +
          + /** Preallocated byte buffer for writing single characters. */
          + private final byte[] singleCharWrite = new byte[1];
          +
          + /** Multipart upload details; null means none started. */
          + private MultiPartUpload multiPartUpload;
          +
          + /** Closed flag. */
          + private final AtomicBoolean closed = new AtomicBoolean(false);
          +
          + /** Current data block. Null means none currently active */
          + private S3ADataBlocks.DataBlock activeBlock;
          +
          + /** Count of blocks uploaded. */
          + private long blockCount = 0;
          +
          + /** Statistics to build up. */
          + private final S3AInstrumentation.OutputStreamStatistics statistics;
          +
          + /**
          + * Write operation helper; encapsulation of the filesystem operations.
          + */
          + private final S3AFileSystem.WriteOperationHelper writeOperationHelper;
          +
          + /**
          + * An S3A output stream which uploads partitions in a separate pool of
          + * threads; different

          {@link S3ADataBlocks.BlockFactory}

          + * instances can control where data is buffered.
          + *
          + * @param fs S3AFilesystem
          + * @param key S3 object to work on.
          + * @param executorService the executor service to use to schedule work
          + * @param progress report progress in order to prevent timeouts. If
          + * this class implements

          {@code ProgressListener}

          then it will be
          — End diff –

          correct. your diligence in reading javadocs is appreciated

          Show
          githubbot ASF GitHub Bot added a comment - Github user steveloughran commented on a diff in the pull request: https://github.com/apache/hadoop/pull/130#discussion_r83244207 — Diff: hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java — @@ -0,0 +1,699 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.s3a; + +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.Callable; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; + +import com.amazonaws.AmazonClientException; +import com.amazonaws.event.ProgressEvent; +import com.amazonaws.event.ProgressEventType; +import com.amazonaws.event.ProgressListener; +import com.amazonaws.services.s3.model.CompleteMultipartUploadResult; +import com.amazonaws.services.s3.model.PartETag; +import com.amazonaws.services.s3.model.PutObjectRequest; +import com.amazonaws.services.s3.model.PutObjectResult; +import com.amazonaws.services.s3.model.UploadPartRequest; +import com.google.common.base.Preconditions; +import com.google.common.util.concurrent.Futures; +import com.google.common.util.concurrent.ListenableFuture; +import com.google.common.util.concurrent.ListeningExecutorService; +import com.google.common.util.concurrent.MoreExecutors; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.io.IOUtils; +import org.apache.hadoop.io.retry.RetryPolicies; +import org.apache.hadoop.io.retry.RetryPolicy; +import org.apache.hadoop.util.Progressable; + +import static org.apache.hadoop.fs.s3a.S3AUtils.*; +import static org.apache.hadoop.fs.s3a.Statistic.*; + +/** + * Upload files/parts directly via different buffering mechanisms: + * including memory and disk. + * + * If the stream is closed and no update has started, then the upload + * is instead done as a single PUT operation. + * + * Unstable: statistics and error handling might evolve. + */ +@InterfaceAudience.Private +@InterfaceStability.Unstable +class S3ABlockOutputStream extends OutputStream { + + private static final Logger LOG = + LoggerFactory.getLogger(S3ABlockOutputStream.class); + + /** Owner FileSystem. */ + private final S3AFileSystem fs; + + /** Object being uploaded. */ + private final String key; + + /** Size of all blocks. */ + private final int blockSize; + + /** Callback for progress. */ + private final ProgressListener progressListener; + private final ListeningExecutorService executorService; + + /** + * Retry policy for multipart commits; not all AWS SDK versions retry that. + */ + private final RetryPolicy retryPolicy = + RetryPolicies.retryUpToMaximumCountWithProportionalSleep( + 5, + 2000, + TimeUnit.MILLISECONDS); + /** + * Factory for blocks. + */ + private final S3ADataBlocks.BlockFactory blockFactory; + + /** Preallocated byte buffer for writing single characters. */ + private final byte[] singleCharWrite = new byte [1] ; + + /** Multipart upload details; null means none started. */ + private MultiPartUpload multiPartUpload; + + /** Closed flag. */ + private final AtomicBoolean closed = new AtomicBoolean(false); + + /** Current data block. Null means none currently active */ + private S3ADataBlocks.DataBlock activeBlock; + + /** Count of blocks uploaded. */ + private long blockCount = 0; + + /** Statistics to build up. */ + private final S3AInstrumentation.OutputStreamStatistics statistics; + + /** + * Write operation helper; encapsulation of the filesystem operations. + */ + private final S3AFileSystem.WriteOperationHelper writeOperationHelper; + + /** + * An S3A output stream which uploads partitions in a separate pool of + * threads; different {@link S3ADataBlocks.BlockFactory} + * instances can control where data is buffered. + * + * @param fs S3AFilesystem + * @param key S3 object to work on. + * @param executorService the executor service to use to schedule work + * @param progress report progress in order to prevent timeouts. If + * this class implements {@code ProgressListener} then it will be — End diff – correct. your diligence in reading javadocs is appreciated
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user steveloughran commented on a diff in the pull request:

          https://github.com/apache/hadoop/pull/130#discussion_r83244375

          — Diff: hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java —
          @@ -0,0 +1,699 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +
          +package org.apache.hadoop.fs.s3a;
          +
          +import java.io.IOException;
          +import java.io.InputStream;
          +import java.io.OutputStream;
          +import java.util.ArrayList;
          +import java.util.List;
          +import java.util.concurrent.Callable;
          +import java.util.concurrent.ExecutionException;
          +import java.util.concurrent.ExecutorService;
          +import java.util.concurrent.TimeUnit;
          +import java.util.concurrent.atomic.AtomicBoolean;
          +
          +import com.amazonaws.AmazonClientException;
          +import com.amazonaws.event.ProgressEvent;
          +import com.amazonaws.event.ProgressEventType;
          +import com.amazonaws.event.ProgressListener;
          +import com.amazonaws.services.s3.model.CompleteMultipartUploadResult;
          +import com.amazonaws.services.s3.model.PartETag;
          +import com.amazonaws.services.s3.model.PutObjectRequest;
          +import com.amazonaws.services.s3.model.PutObjectResult;
          +import com.amazonaws.services.s3.model.UploadPartRequest;
          +import com.google.common.base.Preconditions;
          +import com.google.common.util.concurrent.Futures;
          +import com.google.common.util.concurrent.ListenableFuture;
          +import com.google.common.util.concurrent.ListeningExecutorService;
          +import com.google.common.util.concurrent.MoreExecutors;
          +import org.slf4j.Logger;
          +import org.slf4j.LoggerFactory;
          +
          +import org.apache.hadoop.classification.InterfaceAudience;
          +import org.apache.hadoop.classification.InterfaceStability;
          +import org.apache.hadoop.io.IOUtils;
          +import org.apache.hadoop.io.retry.RetryPolicies;
          +import org.apache.hadoop.io.retry.RetryPolicy;
          +import org.apache.hadoop.util.Progressable;
          +
          +import static org.apache.hadoop.fs.s3a.S3AUtils.*;
          +import static org.apache.hadoop.fs.s3a.Statistic.*;
          +
          +/**
          + * Upload files/parts directly via different buffering mechanisms:
          + * including memory and disk.
          + *
          + * If the stream is closed and no update has started, then the upload
          + * is instead done as a single PUT operation.
          + *
          + * Unstable: statistics and error handling might evolve.
          + */
          +@InterfaceAudience.Private
          +@InterfaceStability.Unstable
          +class S3ABlockOutputStream extends OutputStream {
          +
          + private static final Logger LOG =
          + LoggerFactory.getLogger(S3ABlockOutputStream.class);
          +
          + /** Owner FileSystem. */
          + private final S3AFileSystem fs;
          +
          + /** Object being uploaded. */
          + private final String key;
          +
          + /** Size of all blocks. */
          + private final int blockSize;
          +
          + /** Callback for progress. */
          + private final ProgressListener progressListener;
          + private final ListeningExecutorService executorService;
          +
          + /**
          + * Retry policy for multipart commits; not all AWS SDK versions retry that.
          + */
          + private final RetryPolicy retryPolicy =
          + RetryPolicies.retryUpToMaximumCountWithProportionalSleep(
          + 5,
          + 2000,
          + TimeUnit.MILLISECONDS);
          + /**
          + * Factory for blocks.
          + */
          + private final S3ADataBlocks.BlockFactory blockFactory;
          +
          + /** Preallocated byte buffer for writing single characters. */
          + private final byte[] singleCharWrite = new byte[1];
          +
          + /** Multipart upload details; null means none started. */
          + private MultiPartUpload multiPartUpload;
          +
          + /** Closed flag. */
          + private final AtomicBoolean closed = new AtomicBoolean(false);
          +
          + /** Current data block. Null means none currently active */
          + private S3ADataBlocks.DataBlock activeBlock;
          +
          + /** Count of blocks uploaded. */
          + private long blockCount = 0;
          +
          + /** Statistics to build up. */
          + private final S3AInstrumentation.OutputStreamStatistics statistics;
          +
          + /**
          + * Write operation helper; encapsulation of the filesystem operations.
          + */
          + private final S3AFileSystem.WriteOperationHelper writeOperationHelper;
          +
          + /**
          + * An S3A output stream which uploads partitions in a separate pool of
          + * threads; different

          {@link S3ADataBlocks.BlockFactory}

          + * instances can control where data is buffered.
          + *
          + * @param fs S3AFilesystem
          + * @param key S3 object to work on.
          + * @param executorService the executor service to use to schedule work
          + * @param progress report progress in order to prevent timeouts. If
          + * this class implements

          {@code ProgressListener}

          then it will be
          + * directly wired up to the AWS client, so receive detailed progress
          + * information.
          + * @param blockSize size of a single block.
          + * @param blockFactory factory for creating stream destinations
          + * @param statistics stats for this stream
          + * @param writeOperationHelper state of the write operation.
          + * @throws IOException on any problem
          + */
          + S3ABlockOutputStream(S3AFileSystem fs,
          + String key,
          + ExecutorService executorService,
          + Progressable progress,
          + long blockSize,
          + S3ADataBlocks.BlockFactory blockFactory,
          + S3AInstrumentation.OutputStreamStatistics statistics,
          + S3AFileSystem.WriteOperationHelper writeOperationHelper)
          + throws IOException {
          + this.fs = fs;
          + this.key = key;
          + this.blockFactory = blockFactory;
          + this.blockSize = (int) blockSize;
          + this.statistics = statistics;
          + this.writeOperationHelper = writeOperationHelper;
          + Preconditions.checkArgument(blockSize >= Constants.MULTIPART_MIN_SIZE,
          + "Block size is too small: %d", blockSize);
          + this.executorService = MoreExecutors.listeningDecorator(executorService);
          + this.multiPartUpload = null;
          + this.progressListener = (progress instanceof ProgressListener) ?
          + (ProgressListener) progress
          + : new ProgressableListener(progress);
          + // create that first block. This guarantees that an open + close sequence
          + // writes a 0-byte entry.
          + maybeCreateBlock();
          + LOG.debug("Initialized S3ABlockOutputStream for {}" +
          + " output to {}", writeOperationHelper, activeBlock);
          + }
          +
          + /**
          + * Demand create a destination block.
          + * @return the active block; null if there isn't one.
          + * @throws IOException on any failure to create
          + */
          + private synchronized S3ADataBlocks.DataBlock maybeCreateBlock()
          — End diff –

          renamed

          Show
          githubbot ASF GitHub Bot added a comment - Github user steveloughran commented on a diff in the pull request: https://github.com/apache/hadoop/pull/130#discussion_r83244375 — Diff: hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java — @@ -0,0 +1,699 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.s3a; + +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.Callable; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; + +import com.amazonaws.AmazonClientException; +import com.amazonaws.event.ProgressEvent; +import com.amazonaws.event.ProgressEventType; +import com.amazonaws.event.ProgressListener; +import com.amazonaws.services.s3.model.CompleteMultipartUploadResult; +import com.amazonaws.services.s3.model.PartETag; +import com.amazonaws.services.s3.model.PutObjectRequest; +import com.amazonaws.services.s3.model.PutObjectResult; +import com.amazonaws.services.s3.model.UploadPartRequest; +import com.google.common.base.Preconditions; +import com.google.common.util.concurrent.Futures; +import com.google.common.util.concurrent.ListenableFuture; +import com.google.common.util.concurrent.ListeningExecutorService; +import com.google.common.util.concurrent.MoreExecutors; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.io.IOUtils; +import org.apache.hadoop.io.retry.RetryPolicies; +import org.apache.hadoop.io.retry.RetryPolicy; +import org.apache.hadoop.util.Progressable; + +import static org.apache.hadoop.fs.s3a.S3AUtils.*; +import static org.apache.hadoop.fs.s3a.Statistic.*; + +/** + * Upload files/parts directly via different buffering mechanisms: + * including memory and disk. + * + * If the stream is closed and no update has started, then the upload + * is instead done as a single PUT operation. + * + * Unstable: statistics and error handling might evolve. + */ +@InterfaceAudience.Private +@InterfaceStability.Unstable +class S3ABlockOutputStream extends OutputStream { + + private static final Logger LOG = + LoggerFactory.getLogger(S3ABlockOutputStream.class); + + /** Owner FileSystem. */ + private final S3AFileSystem fs; + + /** Object being uploaded. */ + private final String key; + + /** Size of all blocks. */ + private final int blockSize; + + /** Callback for progress. */ + private final ProgressListener progressListener; + private final ListeningExecutorService executorService; + + /** + * Retry policy for multipart commits; not all AWS SDK versions retry that. + */ + private final RetryPolicy retryPolicy = + RetryPolicies.retryUpToMaximumCountWithProportionalSleep( + 5, + 2000, + TimeUnit.MILLISECONDS); + /** + * Factory for blocks. + */ + private final S3ADataBlocks.BlockFactory blockFactory; + + /** Preallocated byte buffer for writing single characters. */ + private final byte[] singleCharWrite = new byte [1] ; + + /** Multipart upload details; null means none started. */ + private MultiPartUpload multiPartUpload; + + /** Closed flag. */ + private final AtomicBoolean closed = new AtomicBoolean(false); + + /** Current data block. Null means none currently active */ + private S3ADataBlocks.DataBlock activeBlock; + + /** Count of blocks uploaded. */ + private long blockCount = 0; + + /** Statistics to build up. */ + private final S3AInstrumentation.OutputStreamStatistics statistics; + + /** + * Write operation helper; encapsulation of the filesystem operations. + */ + private final S3AFileSystem.WriteOperationHelper writeOperationHelper; + + /** + * An S3A output stream which uploads partitions in a separate pool of + * threads; different {@link S3ADataBlocks.BlockFactory} + * instances can control where data is buffered. + * + * @param fs S3AFilesystem + * @param key S3 object to work on. + * @param executorService the executor service to use to schedule work + * @param progress report progress in order to prevent timeouts. If + * this class implements {@code ProgressListener} then it will be + * directly wired up to the AWS client, so receive detailed progress + * information. + * @param blockSize size of a single block. + * @param blockFactory factory for creating stream destinations + * @param statistics stats for this stream + * @param writeOperationHelper state of the write operation. + * @throws IOException on any problem + */ + S3ABlockOutputStream(S3AFileSystem fs, + String key, + ExecutorService executorService, + Progressable progress, + long blockSize, + S3ADataBlocks.BlockFactory blockFactory, + S3AInstrumentation.OutputStreamStatistics statistics, + S3AFileSystem.WriteOperationHelper writeOperationHelper) + throws IOException { + this.fs = fs; + this.key = key; + this.blockFactory = blockFactory; + this.blockSize = (int) blockSize; + this.statistics = statistics; + this.writeOperationHelper = writeOperationHelper; + Preconditions.checkArgument(blockSize >= Constants.MULTIPART_MIN_SIZE, + "Block size is too small: %d", blockSize); + this.executorService = MoreExecutors.listeningDecorator(executorService); + this.multiPartUpload = null; + this.progressListener = (progress instanceof ProgressListener) ? + (ProgressListener) progress + : new ProgressableListener(progress); + // create that first block. This guarantees that an open + close sequence + // writes a 0-byte entry. + maybeCreateBlock(); + LOG.debug("Initialized S3ABlockOutputStream for {}" + + " output to {}", writeOperationHelper, activeBlock); + } + + /** + * Demand create a destination block. + * @return the active block; null if there isn't one. + * @throws IOException on any failure to create + */ + private synchronized S3ADataBlocks.DataBlock maybeCreateBlock() — End diff – renamed
          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 24 new or modified test files.
          0 mvndep 0m 54s Maven dependency ordering for branch
          +1 mvninstall 6m 57s branch-2 passed
          +1 compile 6m 39s branch-2 passed with JDK v1.8.0_101
          +1 compile 6m 41s branch-2 passed with JDK v1.7.0_111
          +1 checkstyle 1m 28s branch-2 passed
          +1 mvnsite 1m 21s branch-2 passed
          +1 mvneclipse 0m 35s branch-2 passed
          +1 findbugs 2m 15s branch-2 passed
          +1 javadoc 1m 2s branch-2 passed with JDK v1.8.0_101
          +1 javadoc 1m 15s branch-2 passed with JDK v1.7.0_111
          0 mvndep 0m 15s Maven dependency ordering for patch
          +1 mvninstall 1m 1s the patch passed
          +1 compile 6m 44s the patch passed with JDK v1.8.0_101
          -1 javac 6m 44s root-jdk1.8.0_101 with JDK v1.8.0_101 generated 1 new + 857 unchanged - 0 fixed = 858 total (was 857)
          +1 compile 6m 44s the patch passed with JDK v1.7.0_111
          -1 javac 6m 44s root-jdk1.7.0_111 with JDK v1.7.0_111 generated 2 new + 949 unchanged - 1 fixed = 951 total (was 950)
          -0 checkstyle 1m 29s root: The patch generated 26 new + 48 unchanged - 2 fixed = 74 total (was 50)
          +1 mvnsite 1m 27s the patch passed
          +1 mvneclipse 0m 36s the patch passed
          -1 whitespace 0m 0s The patch has 62 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 2m 42s the patch passed
          +1 javadoc 1m 8s the patch passed with JDK v1.8.0_101
          +1 javadoc 1m 20s the patch passed with JDK v1.7.0_111
          -1 unit 7m 35s hadoop-common in the patch failed with JDK v1.7.0_111.
          +1 unit 0m 28s hadoop-aws in the patch passed with JDK v1.7.0_111.
          +1 asflicense 0m 29s The patch does not generate ASF License warnings.
          95m 22s



          Reason Tests
          JDK v1.7.0_111 Failed junit tests hadoop.net.TestDNS



          Subsystem Report/Notes
          Docker Image:yetus/hadoop:b59b8b7
          JIRA Issue HADOOP-13560
          GITHUB PR https://github.com/apache/hadoop/pull/130
          Optional Tests asflicense compile javac javadoc mvninstall mvnsite unit xml findbugs checkstyle
          uname Linux e56d47fe6748 3.13.0-93-generic #140-Ubuntu SMP Mon Jul 18 21:21:05 UTC 2016 x86_64 x86_64 x86_64 GNU/Linux
          Build tool maven
          Personality /testptch/hadoop/patchprocess/precommit/personality/provided.sh
          git revision branch-2 / ad69baf
          Default Java 1.7.0_111
          Multi-JDK versions /usr/lib/jvm/java-8-oracle:1.8.0_101 /usr/lib/jvm/java-7-openjdk-amd64:1.7.0_111
          findbugs v3.0.0
          javac https://builds.apache.org/job/PreCommit-HADOOP-Build/10773/artifact/patchprocess/diff-compile-javac-root-jdk1.8.0_101.txt
          javac https://builds.apache.org/job/PreCommit-HADOOP-Build/10773/artifact/patchprocess/diff-compile-javac-root-jdk1.7.0_111.txt
          checkstyle https://builds.apache.org/job/PreCommit-HADOOP-Build/10773/artifact/patchprocess/diff-checkstyle-root.txt
          whitespace https://builds.apache.org/job/PreCommit-HADOOP-Build/10773/artifact/patchprocess/whitespace-eol.txt
          unit https://builds.apache.org/job/PreCommit-HADOOP-Build/10773/artifact/patchprocess/patch-unit-hadoop-common-project_hadoop-common-jdk1.7.0_111.txt
          JDK v1.7.0_111 Test Results https://builds.apache.org/job/PreCommit-HADOOP-Build/10773/testReport/
          modules C: hadoop-common-project/hadoop-common hadoop-tools/hadoop-aws U: .
          Console output https://builds.apache.org/job/PreCommit-HADOOP-Build/10773/console
          Powered by Apache Yetus 0.4.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 24 new or modified test files. 0 mvndep 0m 54s Maven dependency ordering for branch +1 mvninstall 6m 57s branch-2 passed +1 compile 6m 39s branch-2 passed with JDK v1.8.0_101 +1 compile 6m 41s branch-2 passed with JDK v1.7.0_111 +1 checkstyle 1m 28s branch-2 passed +1 mvnsite 1m 21s branch-2 passed +1 mvneclipse 0m 35s branch-2 passed +1 findbugs 2m 15s branch-2 passed +1 javadoc 1m 2s branch-2 passed with JDK v1.8.0_101 +1 javadoc 1m 15s branch-2 passed with JDK v1.7.0_111 0 mvndep 0m 15s Maven dependency ordering for patch +1 mvninstall 1m 1s the patch passed +1 compile 6m 44s the patch passed with JDK v1.8.0_101 -1 javac 6m 44s root-jdk1.8.0_101 with JDK v1.8.0_101 generated 1 new + 857 unchanged - 0 fixed = 858 total (was 857) +1 compile 6m 44s the patch passed with JDK v1.7.0_111 -1 javac 6m 44s root-jdk1.7.0_111 with JDK v1.7.0_111 generated 2 new + 949 unchanged - 1 fixed = 951 total (was 950) -0 checkstyle 1m 29s root: The patch generated 26 new + 48 unchanged - 2 fixed = 74 total (was 50) +1 mvnsite 1m 27s the patch passed +1 mvneclipse 0m 36s the patch passed -1 whitespace 0m 0s The patch has 62 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 2m 42s the patch passed +1 javadoc 1m 8s the patch passed with JDK v1.8.0_101 +1 javadoc 1m 20s the patch passed with JDK v1.7.0_111 -1 unit 7m 35s hadoop-common in the patch failed with JDK v1.7.0_111. +1 unit 0m 28s hadoop-aws in the patch passed with JDK v1.7.0_111. +1 asflicense 0m 29s The patch does not generate ASF License warnings. 95m 22s Reason Tests JDK v1.7.0_111 Failed junit tests hadoop.net.TestDNS Subsystem Report/Notes Docker Image:yetus/hadoop:b59b8b7 JIRA Issue HADOOP-13560 GITHUB PR https://github.com/apache/hadoop/pull/130 Optional Tests asflicense compile javac javadoc mvninstall mvnsite unit xml findbugs checkstyle uname Linux e56d47fe6748 3.13.0-93-generic #140-Ubuntu SMP Mon Jul 18 21:21:05 UTC 2016 x86_64 x86_64 x86_64 GNU/Linux Build tool maven Personality /testptch/hadoop/patchprocess/precommit/personality/provided.sh git revision branch-2 / ad69baf Default Java 1.7.0_111 Multi-JDK versions /usr/lib/jvm/java-8-oracle:1.8.0_101 /usr/lib/jvm/java-7-openjdk-amd64:1.7.0_111 findbugs v3.0.0 javac https://builds.apache.org/job/PreCommit-HADOOP-Build/10773/artifact/patchprocess/diff-compile-javac-root-jdk1.8.0_101.txt javac https://builds.apache.org/job/PreCommit-HADOOP-Build/10773/artifact/patchprocess/diff-compile-javac-root-jdk1.7.0_111.txt checkstyle https://builds.apache.org/job/PreCommit-HADOOP-Build/10773/artifact/patchprocess/diff-checkstyle-root.txt whitespace https://builds.apache.org/job/PreCommit-HADOOP-Build/10773/artifact/patchprocess/whitespace-eol.txt unit https://builds.apache.org/job/PreCommit-HADOOP-Build/10773/artifact/patchprocess/patch-unit-hadoop-common-project_hadoop-common-jdk1.7.0_111.txt JDK v1.7.0_111 Test Results https://builds.apache.org/job/PreCommit-HADOOP-Build/10773/testReport/ modules C: hadoop-common-project/hadoop-common hadoop-tools/hadoop-aws U: . Console output https://builds.apache.org/job/PreCommit-HADOOP-Build/10773/console Powered by Apache Yetus 0.4.0-SNAPSHOT http://yetus.apache.org This message was automatically generated.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user thodemoor commented on a diff in the pull request:

          https://github.com/apache/hadoop/pull/130#discussion_r83384132

          — Diff: hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md —
          @@ -881,40 +881,362 @@ Seoul
          If the wrong endpoint is used, the request may fail. This may be reported as a 301/redirect error,
          or as a 400 Bad Request.

                1. S3AFastOutputStream
          • *Warning: NEW in hadoop 2.7. UNSTABLE, EXPERIMENTAL: use at own risk*
          • <property>
          • <name>fs.s3a.fast.upload</name>
          • <value>false</value>
          • <description>Upload directly from memory instead of buffering to
          • disk first. Memory usage and parallelism can be controlled as up to
          • fs.s3a.multipart.size memory is consumed for each (part)upload actively
          • uploading (fs.s3a.threads.max) or queueing (fs.s3a.max.total.tasks)</description>
          • </property>
          • <property>
          • <name>fs.s3a.fast.buffer.size</name>
          • <value>1048576</value>
          • <description>Size (in bytes) of initial memory buffer allocated for an
          • upload. No effect if fs.s3a.fast.upload is false.</description>
          • </property>
            +### <a name="s3a_fast_upload"></a>Stabilizing: S3A Fast Upload
            +
            +
            +*New in Hadoop 2.7; significantly enhanced in Hadoop 2.9*
            +
            +
            +Because of the nature of the S3 object store, data written to an S3A `OutputStream`
            +is not written incrementally —instead, by default, it is buffered to disk
            +until the stream is closed in its `close()` method.
            +
            +This can make output slow:
            +
            +* The execution time for `OutputStream.close()` is proportional to the amount of data
            +buffered and inversely proportional to the bandwidth. That is `O(data/bandwidth)`.
            +* The bandwidth is that available from the host to S3: other work in the same
            +process, server or network at the time of upload may increase the upload time,
            +hence the duration of the `close()` call.
            +* If a process uploading data fails before `OutputStream.close()` is called,
            +all data is lost.
            +* The disks hosting temporary directories defined in `fs.s3a.buffer.dir` must
            +have the capacity to store the entire buffered file.
            +
            +Put succinctly: the further the process is from the S3 endpoint, or the smaller
            +the EC-hosted VM is, the longer it will take work to complete.
            +
            +This can create problems in application code:
            +
            +* Code often assumes that the `close()` call is fast;
            + the delays can create bottlenecks in operations.
            +* Very slow uploads sometimes cause applications to time out. (generally,
            +threads blocking during the upload stop reporting progress, so trigger timeouts)
            +* Streaming very large amounts of data may consume all disk space before the upload begins.
            +
            +
            +Work to addess this began in Hadoop 2.7 with the `S3AFastOutputStream`
            +HADOOP-11183(https://issues.apache.org/jira/browse/HADOOP-11183), and
            +has continued with ` S3ABlockOutputStream`
            +HADOOP-13560(https://issues.apache.org/jira/browse/HADOOP-13560).
            +
            +
            +This adds an alternative output stream, "S3a Fast Upload" which:
            +
            +1. Always uploads large files as blocks with the size set by
            + `fs.s3a.multipart.size`. That is: the threshold at which multipart uploads
            + begin and the size of each upload are identical.
            +1. Buffers blocks to disk (default) or in on-heap or off-heap memory.
            +1. Uploads blocks in parallel in background threads.
            +1. Begins uploading blocks as soon as the buffered data exceeds this partition
            + size.
            +1. When buffering data to disk, uses the directory/directories listed in
            + `fs.s3a.buffer.dir`. The size of data which can be buffered is limited
            + to the available disk space.
            +1. Generates output statistics as metrics on the filesystem, including
            + statistics of active and pending block uploads.
            +1. Has the time to `close()` set by the amount of remaning data to upload, rather
            + than the total size of the file.
            +
            +With incremental writes of blocks, "S3A fast upload" offers an upload
            +time at least as fast as the "classic" mechanism, with significant benefits
            +on long-lived output streams, and when very large amounts of data are generated.
            +The in memory buffering mechanims may also offer speedup when running adjacent to
            +S3 endpoints, as disks are not used for intermediate data storage.
            +
            +
            +```xml
            +<property>
            + <name>fs.s3a.fast.upload</name>
            + <value>true</value>
            + <description>
            + Use the incremental block upload mechanism with
            + the buffering mechanism set in fs.s3a.fast.upload.buffer.
            + The number of threads performing uploads in the filesystem is defined
            + by fs.s3a.threads.max; the queue of waiting uploads limited by
            + fs.s3a.max.total.tasks.
            + The size of each buffer is set by fs.s3a.multipart.size.
            + </description>
            +</property>
            +
            +<property>
            + <name>fs.s3a.fast.upload.buffer</name>
            + <value>disk</value>
            + <description>
            + The buffering mechanism to use when using S3A fast upload
            + (fs.s3a.fast.upload=true). Values: disk, array, bytebuffer.
            + This configuration option has no effect if fs.s3a.fast.upload is false.
            +
            + "disk" will use the directories listed in fs.s3a.buffer.dir as
            + the location(s) to save data prior to being uploaded.
            +
            + "array" uses arrays in the JVM heap
            +
            + "bytebuffer" uses off-heap memory within the JVM.
            +
            + Both "array" and "bytebuffer" will consume memory in a single stream up to the number
            + of blocks set by:
            +
            + fs.s3a.multipart.size * fs.s3a.fast.upload.active.blocks.
            +
            + If using either of these mechanisms, keep this value low
            +
            + The total number of threads performing work across all threads is set by
            + fs.s3a.threads.max, with fs.s3a.max.total.tasks values setting the number of queued
            + work items.
              • End diff –

          idem as in pom.xml

          Show
          githubbot ASF GitHub Bot added a comment - Github user thodemoor commented on a diff in the pull request: https://github.com/apache/hadoop/pull/130#discussion_r83384132 — Diff: hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md — @@ -881,40 +881,362 @@ Seoul If the wrong endpoint is used, the request may fail. This may be reported as a 301/redirect error, or as a 400 Bad Request. S3AFastOutputStream * Warning: NEW in hadoop 2.7. UNSTABLE, EXPERIMENTAL: use at own risk * <property> <name>fs.s3a.fast.upload</name> <value>false</value> <description>Upload directly from memory instead of buffering to disk first. Memory usage and parallelism can be controlled as up to fs.s3a.multipart.size memory is consumed for each (part)upload actively uploading (fs.s3a.threads.max) or queueing (fs.s3a.max.total.tasks)</description> </property> <property> <name>fs.s3a.fast.buffer.size</name> <value>1048576</value> <description>Size (in bytes) of initial memory buffer allocated for an upload. No effect if fs.s3a.fast.upload is false.</description> </property> +### <a name="s3a_fast_upload"></a>Stabilizing: S3A Fast Upload + + +* New in Hadoop 2.7; significantly enhanced in Hadoop 2.9 * + + +Because of the nature of the S3 object store, data written to an S3A `OutputStream` +is not written incrementally —instead, by default, it is buffered to disk +until the stream is closed in its `close()` method. + +This can make output slow: + +* The execution time for `OutputStream.close()` is proportional to the amount of data +buffered and inversely proportional to the bandwidth. That is `O(data/bandwidth)`. +* The bandwidth is that available from the host to S3: other work in the same +process, server or network at the time of upload may increase the upload time, +hence the duration of the `close()` call. +* If a process uploading data fails before `OutputStream.close()` is called, +all data is lost. +* The disks hosting temporary directories defined in `fs.s3a.buffer.dir` must +have the capacity to store the entire buffered file. + +Put succinctly: the further the process is from the S3 endpoint, or the smaller +the EC-hosted VM is, the longer it will take work to complete. + +This can create problems in application code: + +* Code often assumes that the `close()` call is fast; + the delays can create bottlenecks in operations. +* Very slow uploads sometimes cause applications to time out. (generally, +threads blocking during the upload stop reporting progress, so trigger timeouts) +* Streaming very large amounts of data may consume all disk space before the upload begins. + + +Work to addess this began in Hadoop 2.7 with the `S3AFastOutputStream` + HADOOP-11183 ( https://issues.apache.org/jira/browse/HADOOP-11183 ), and +has continued with ` S3ABlockOutputStream` + HADOOP-13560 ( https://issues.apache.org/jira/browse/HADOOP-13560 ). + + +This adds an alternative output stream, "S3a Fast Upload" which: + +1. Always uploads large files as blocks with the size set by + `fs.s3a.multipart.size`. That is: the threshold at which multipart uploads + begin and the size of each upload are identical. +1. Buffers blocks to disk (default) or in on-heap or off-heap memory. +1. Uploads blocks in parallel in background threads. +1. Begins uploading blocks as soon as the buffered data exceeds this partition + size. +1. When buffering data to disk, uses the directory/directories listed in + `fs.s3a.buffer.dir`. The size of data which can be buffered is limited + to the available disk space. +1. Generates output statistics as metrics on the filesystem, including + statistics of active and pending block uploads. +1. Has the time to `close()` set by the amount of remaning data to upload, rather + than the total size of the file. + +With incremental writes of blocks, "S3A fast upload" offers an upload +time at least as fast as the "classic" mechanism, with significant benefits +on long-lived output streams, and when very large amounts of data are generated. +The in memory buffering mechanims may also offer speedup when running adjacent to +S3 endpoints, as disks are not used for intermediate data storage. + + +```xml +<property> + <name>fs.s3a.fast.upload</name> + <value>true</value> + <description> + Use the incremental block upload mechanism with + the buffering mechanism set in fs.s3a.fast.upload.buffer. + The number of threads performing uploads in the filesystem is defined + by fs.s3a.threads.max; the queue of waiting uploads limited by + fs.s3a.max.total.tasks. + The size of each buffer is set by fs.s3a.multipart.size. + </description> +</property> + +<property> + <name>fs.s3a.fast.upload.buffer</name> + <value>disk</value> + <description> + The buffering mechanism to use when using S3A fast upload + (fs.s3a.fast.upload=true). Values: disk, array, bytebuffer. + This configuration option has no effect if fs.s3a.fast.upload is false. + + "disk" will use the directories listed in fs.s3a.buffer.dir as + the location(s) to save data prior to being uploaded. + + "array" uses arrays in the JVM heap + + "bytebuffer" uses off-heap memory within the JVM. + + Both "array" and "bytebuffer" will consume memory in a single stream up to the number + of blocks set by: + + fs.s3a.multipart.size * fs.s3a.fast.upload.active.blocks. + + If using either of these mechanisms, keep this value low + + The total number of threads performing work across all threads is set by + fs.s3a.threads.max, with fs.s3a.max.total.tasks values setting the number of queued + work items. End diff – idem as in pom.xml
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user thodemoor commented on a diff in the pull request:

          https://github.com/apache/hadoop/pull/130#discussion_r83377114

          — Diff: hadoop-common-project/hadoop-common/src/main/resources/core-default.xml —
          @@ -1095,10 +1102,50 @@
          <property>
          <name>fs.s3a.fast.upload</name>
          <value>false</value>

          • <description>Upload directly from memory instead of buffering to
          • disk first. Memory usage and parallelism can be controlled as up to
          • fs.s3a.multipart.size memory is consumed for each (part)upload actively
          • uploading (fs.s3a.threads.max) or queueing (fs.s3a.max.total.tasks)</description>
            + <description>
            + Use the incremental block-based fast upload mechanism with
            + the buffering mechanism set in fs.s3a.fast.upload.buffer.
            + </description>
            +</property>
            +
            +<property>
            + <name>fs.s3a.fast.upload.buffer</name>
            + <value>disk</value>
            + <description>
            + The buffering mechanism to use when using S3A fast upload
            + (fs.s3a.fast.upload=true). Values: disk, array, bytebuffer.
            + This configuration option has no effect if fs.s3a.fast.upload is false.
            +
            + "disk" will use the directories listed in fs.s3a.buffer.dir as
            + the location(s) to save data prior to being uploaded.
            +
            + "array" uses arrays in the JVM heap
            +
            + "bytebuffer" uses off-heap memory within the JVM.
            +
            + Both "array" and "bytebuffer" will consume memory in a single stream up to the number
            + of blocks set by:
            +
            + fs.s3a.multipart.size * fs.s3a.fast.upload.active.blocks.
            +
            + If using either of these mechanisms, keep this value low
            +
            + The total number of threads performing work across all threads is set by
            + fs.s3a.threads.max, with fs.s3a.max.total.tasks values setting the number of queued
            + work items.
              • End diff –

          The total max block (memory/disk) consumption, across all streams, is bounded by`fs.s3a.multipart.size * ( fs.s3a.fast.upload.active.blocks + fs.s3a.max.total.tasks + 1)` bytes for an instance of S3AFileSystem.

          Show
          githubbot ASF GitHub Bot added a comment - Github user thodemoor commented on a diff in the pull request: https://github.com/apache/hadoop/pull/130#discussion_r83377114 — Diff: hadoop-common-project/hadoop-common/src/main/resources/core-default.xml — @@ -1095,10 +1102,50 @@ <property> <name>fs.s3a.fast.upload</name> <value>false</value> <description>Upload directly from memory instead of buffering to disk first. Memory usage and parallelism can be controlled as up to fs.s3a.multipart.size memory is consumed for each (part)upload actively uploading (fs.s3a.threads.max) or queueing (fs.s3a.max.total.tasks)</description> + <description> + Use the incremental block-based fast upload mechanism with + the buffering mechanism set in fs.s3a.fast.upload.buffer. + </description> +</property> + +<property> + <name>fs.s3a.fast.upload.buffer</name> + <value>disk</value> + <description> + The buffering mechanism to use when using S3A fast upload + (fs.s3a.fast.upload=true). Values: disk, array, bytebuffer. + This configuration option has no effect if fs.s3a.fast.upload is false. + + "disk" will use the directories listed in fs.s3a.buffer.dir as + the location(s) to save data prior to being uploaded. + + "array" uses arrays in the JVM heap + + "bytebuffer" uses off-heap memory within the JVM. + + Both "array" and "bytebuffer" will consume memory in a single stream up to the number + of blocks set by: + + fs.s3a.multipart.size * fs.s3a.fast.upload.active.blocks. + + If using either of these mechanisms, keep this value low + + The total number of threads performing work across all threads is set by + fs.s3a.threads.max, with fs.s3a.max.total.tasks values setting the number of queued + work items. End diff – The total max block (memory/disk) consumption, across all streams, is bounded by`fs.s3a.multipart.size * ( fs.s3a.fast.upload.active.blocks + fs.s3a.max.total.tasks + 1)` bytes for an instance of S3AFileSystem.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user thodemoor commented on a diff in the pull request:

          https://github.com/apache/hadoop/pull/130#discussion_r83381587

          — Diff: hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ADataBlocks.java —
          @@ -0,0 +1,819 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +
          +package org.apache.hadoop.fs.s3a;
          +
          +import java.io.BufferedOutputStream;
          +import java.io.ByteArrayInputStream;
          +import java.io.ByteArrayOutputStream;
          +import java.io.Closeable;
          +import java.io.EOFException;
          +import java.io.File;
          +import java.io.FileInputStream;
          +import java.io.FileNotFoundException;
          +import java.io.FileOutputStream;
          +import java.io.FilterInputStream;
          +import java.io.IOException;
          +import java.io.InputStream;
          +import java.nio.ByteBuffer;
          +import java.util.concurrent.atomic.AtomicBoolean;
          +import java.util.concurrent.atomic.AtomicInteger;
          +
          +import com.google.common.base.Preconditions;
          +import org.slf4j.Logger;
          +import org.slf4j.LoggerFactory;
          +
          +import org.apache.hadoop.fs.FSExceptionMessages;
          +import org.apache.hadoop.util.DirectBufferPool;
          +
          +import static org.apache.hadoop.fs.s3a.S3ADataBlocks.DataBlock.DestState.*;
          +
          +/**
          + * Set of classes to support output streaming into blocks which are then
          + * uploaded as partitions.
          + */
          +final class S3ADataBlocks {
          +
          + private static final Logger LOG =
          + LoggerFactory.getLogger(S3ADataBlocks.class);
          +
          + private S3ADataBlocks()

          { + }

          +
          + /**
          + * Validate args to a write command. These are the same validation checks
          + * expected for any implementation of

          {@code OutputStream.write()}

          .
          + * @param b byte array containing data
          + * @param off offset in array where to start
          + * @param len number of bytes to be written
          + * @throws NullPointerException for a null buffer
          + * @throws IndexOutOfBoundsException if indices are out of range
          + */
          + static void validateWriteArgs(byte[] b, int off, int len)
          + throws IOException {
          + Preconditions.checkNotNull(b);
          + if ((off < 0) || (off > b.length) || (len < 0) ||
          + ((off + len) > b.length) || ((off + len) < 0))

          { + throw new IndexOutOfBoundsException( + "write (b[" + b.length + "], " + off + ", " + len + ')'); + }

          + }
          +
          + /**
          + * Create a factory.
          + * @param owner factory owner
          + * @param name factory name -the option from

          {@link Constants}

          .
          + * @return the factory, ready to be initialized.
          + * @throws IllegalArgumentException if the name is unknown.
          + */
          + static BlockFactory createFactory(S3AFileSystem owner,
          + String name) {
          + switch (name)

          { + case Constants.FAST_UPLOAD_BUFFER_ARRAY: + return new ArrayBlockFactory(owner); + case Constants.FAST_UPLOAD_BUFFER_DISK: + return new DiskBlockFactory(owner); + case Constants.FAST_UPLOAD_BYTEBUFFER: + return new ByteBufferBlockFactory(owner); + default: + throw new IllegalArgumentException("Unsupported block buffer" + + " \"" + name + '"'); + }

          + }
          +
          + /**
          + * Base class for block factories.
          + */
          + static abstract class BlockFactory implements Closeable {
          +
          + private final S3AFileSystem owner;
          +
          + protected BlockFactory(S3AFileSystem owner)

          { + this.owner = owner; + }

          +
          +
          + /**
          + * Create a block.
          + * @param limit limit of the block.
          + * @return a new block.
          + */
          + abstract DataBlock create(int limit) throws IOException;
          +
          + /**
          + * Implement any close/cleanup operation.
          + * Base class is a no-op
          + * @throws IOException -ideally, it shouldn't.
          + */
          + @Override
          + public void close() throws IOException

          { + }

          +
          + /**
          + * Owner.
          + */
          + protected S3AFileSystem getOwner()

          { + return owner; + }

          + }
          +
          + /**
          + * This represents a block being uploaded.
          + */
          + static abstract class DataBlock implements Closeable {
          +
          + enum DestState

          {Writing, Upload, Closed}

          +
          + private volatile DestState state = Writing;
          +
          + /**
          + * Atomically enter a state, verifying current state.
          + * @param current current state. null means "no check"
          + * @param next next state
          + * @throws IllegalStateException if the current state is not as expected
          + */
          + protected synchronized final void enterState(DestState current,
          + DestState next)
          + throws IllegalStateException {
          + verifyState(current);
          + LOG.debug("{}: entering state {}", this, next);
          + state = next;
          + }
          +
          + /**
          + * Verify that the block is in the declared state.
          + * @param expected expected state.
          + * @throws IllegalStateException if the DataBlock is in the wrong state
          + */
          + protected final void verifyState(DestState expected)
          + throws IllegalStateException {
          + if (expected != null && state != expected)

          { + throw new IllegalStateException("Expected stream state " + expected + + " -but actual state is " + state + " in " + this); + }

          + }
          +
          + /**
          + * Current state.
          + * @return the current state.
          + */
          + final DestState getState()

          { + return state; + }

          +
          + /**
          + * Return the current data size.
          + * @return the size of the data
          + */
          + abstract int dataSize();
          +
          + /**
          + * Predicate to verify that the block has the capacity to write
          + * the given set of bytes.
          + * @param bytes number of bytes desired to be written.
          + * @return true if there is enough space.
          + */
          + abstract boolean hasCapacity(long bytes);
          +
          + /**
          + * Predicate to check if there is data in the block.
          + * @return true if there is
          + */
          + boolean hasData()

          { + return dataSize() > 0; + }

          +
          + /**
          + * The remaining capacity in the block before it is full.
          + * @return the number of bytes remaining.
          + */
          + abstract int remainingCapacity();
          +
          + /**
          + * Write a series of bytes from the buffer, from the offset.
          + * Returns the number of bytes written.
          + * Only valid in the state

          {@code Writing}.
          + * Base class verifies the state but does no writing.
          + * @param buffer buffer
          + * @param offset offset
          + * @param length length of write
          + * @return number of bytes written
          + * @throws IOException trouble
          + */
          + int write(byte[] buffer, int offset, int length) throws IOException { + verifyState(Writing); + Preconditions.checkArgument(buffer != null, "Null buffer"); + Preconditions.checkArgument(length >= 0, "length is negative"); + Preconditions.checkArgument(offset >= 0, "offset is negative"); + Preconditions.checkArgument( + !(buffer.length - offset < length), + "buffer shorter than amount of data to write"); + return 0; + }
          +
          + /**
          + * Flush the output.
          + * Only valid in the state {@code Writing}

          .
          + * In the base class, this is a no-op
          + * @throws IOException any IO problem.
          + */
          + void flush() throws IOException

          { + verifyState(Writing); + }

          +
          + /**
          + * Switch to the upload state and return a stream for uploading.
          + * Base class calls

          {@link #enterState(DestState, DestState)}

          to
          + * manage the state machine.
          + * @return the stream
          + * @throws IOException trouble
          + */
          + InputStream startUpload() throws IOException

          { + LOG.debug("Start datablock upload"); + enterState(Writing, Upload); + return null; + }

          +
          + /**
          + * Enter the closed state.
          + * @return true if the class was in any other state, implying that
          + * the subclass should do its close operations
          + */
          + protected synchronized boolean enterClosedState() {
          + if (!state.equals(Closed))

          { + enterState(null, Closed); + return true; + }

          else

          { + return false; + }

          + }
          +
          + @Override
          + public void close() throws IOException {
          + if (enterClosedState()) {
          + LOG.debug("Closed {}", this);
          + innerClose();
          + }
          + }
          +
          + /**
          + * Inner close logic for subclasses to implement.
          + */
          + protected void innerClose() throws IOException

          { + + }

          +
          + }
          +
          + // ====================================================================
          +
          + /**
          + * Use byte arrays on the heap for storage.
          + */
          + static class ArrayBlockFactory extends BlockFactory {
          +
          + ArrayBlockFactory(S3AFileSystem owner)

          { + super(owner); + }

          +
          + @Override
          + DataBlock create(int limit) throws IOException

          { + return new ByteArrayBlock(limit); + }

          +
          + }
          +
          + /**
          + * Stream to memory via a

          {@code ByteArrayOutputStream}

          .
          + *
          + * This was taken from

          {@code S3AFastOutputStream}

          and has the
          + * same problem which surfaced there: it consumes heap space
          + * proportional to the mismatch between writes to the stream and
          + * the JVM-wide upload bandwidth to the S3 endpoint.
          — End diff –

          but bounded by ...

          Show
          githubbot ASF GitHub Bot added a comment - Github user thodemoor commented on a diff in the pull request: https://github.com/apache/hadoop/pull/130#discussion_r83381587 — Diff: hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ADataBlocks.java — @@ -0,0 +1,819 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.s3a; + +import java.io.BufferedOutputStream; +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.Closeable; +import java.io.EOFException; +import java.io.File; +import java.io.FileInputStream; +import java.io.FileNotFoundException; +import java.io.FileOutputStream; +import java.io.FilterInputStream; +import java.io.IOException; +import java.io.InputStream; +import java.nio.ByteBuffer; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; + +import com.google.common.base.Preconditions; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.fs.FSExceptionMessages; +import org.apache.hadoop.util.DirectBufferPool; + +import static org.apache.hadoop.fs.s3a.S3ADataBlocks.DataBlock.DestState.*; + +/** + * Set of classes to support output streaming into blocks which are then + * uploaded as partitions. + */ +final class S3ADataBlocks { + + private static final Logger LOG = + LoggerFactory.getLogger(S3ADataBlocks.class); + + private S3ADataBlocks() { + } + + /** + * Validate args to a write command. These are the same validation checks + * expected for any implementation of {@code OutputStream.write()} . + * @param b byte array containing data + * @param off offset in array where to start + * @param len number of bytes to be written + * @throws NullPointerException for a null buffer + * @throws IndexOutOfBoundsException if indices are out of range + */ + static void validateWriteArgs(byte[] b, int off, int len) + throws IOException { + Preconditions.checkNotNull(b); + if ((off < 0) || (off > b.length) || (len < 0) || + ((off + len) > b.length) || ((off + len) < 0)) { + throw new IndexOutOfBoundsException( + "write (b[" + b.length + "], " + off + ", " + len + ')'); + } + } + + /** + * Create a factory. + * @param owner factory owner + * @param name factory name -the option from {@link Constants} . + * @return the factory, ready to be initialized. + * @throws IllegalArgumentException if the name is unknown. + */ + static BlockFactory createFactory(S3AFileSystem owner, + String name) { + switch (name) { + case Constants.FAST_UPLOAD_BUFFER_ARRAY: + return new ArrayBlockFactory(owner); + case Constants.FAST_UPLOAD_BUFFER_DISK: + return new DiskBlockFactory(owner); + case Constants.FAST_UPLOAD_BYTEBUFFER: + return new ByteBufferBlockFactory(owner); + default: + throw new IllegalArgumentException("Unsupported block buffer" + + " \"" + name + '"'); + } + } + + /** + * Base class for block factories. + */ + static abstract class BlockFactory implements Closeable { + + private final S3AFileSystem owner; + + protected BlockFactory(S3AFileSystem owner) { + this.owner = owner; + } + + + /** + * Create a block. + * @param limit limit of the block. + * @return a new block. + */ + abstract DataBlock create(int limit) throws IOException; + + /** + * Implement any close/cleanup operation. + * Base class is a no-op + * @throws IOException -ideally, it shouldn't. + */ + @Override + public void close() throws IOException { + } + + /** + * Owner. + */ + protected S3AFileSystem getOwner() { + return owner; + } + } + + /** + * This represents a block being uploaded. + */ + static abstract class DataBlock implements Closeable { + + enum DestState {Writing, Upload, Closed} + + private volatile DestState state = Writing; + + /** + * Atomically enter a state, verifying current state. + * @param current current state. null means "no check" + * @param next next state + * @throws IllegalStateException if the current state is not as expected + */ + protected synchronized final void enterState(DestState current, + DestState next) + throws IllegalStateException { + verifyState(current); + LOG.debug("{}: entering state {}", this, next); + state = next; + } + + /** + * Verify that the block is in the declared state. + * @param expected expected state. + * @throws IllegalStateException if the DataBlock is in the wrong state + */ + protected final void verifyState(DestState expected) + throws IllegalStateException { + if (expected != null && state != expected) { + throw new IllegalStateException("Expected stream state " + expected + + " -but actual state is " + state + " in " + this); + } + } + + /** + * Current state. + * @return the current state. + */ + final DestState getState() { + return state; + } + + /** + * Return the current data size. + * @return the size of the data + */ + abstract int dataSize(); + + /** + * Predicate to verify that the block has the capacity to write + * the given set of bytes. + * @param bytes number of bytes desired to be written. + * @return true if there is enough space. + */ + abstract boolean hasCapacity(long bytes); + + /** + * Predicate to check if there is data in the block. + * @return true if there is + */ + boolean hasData() { + return dataSize() > 0; + } + + /** + * The remaining capacity in the block before it is full. + * @return the number of bytes remaining. + */ + abstract int remainingCapacity(); + + /** + * Write a series of bytes from the buffer, from the offset. + * Returns the number of bytes written. + * Only valid in the state {@code Writing}. + * Base class verifies the state but does no writing. + * @param buffer buffer + * @param offset offset + * @param length length of write + * @return number of bytes written + * @throws IOException trouble + */ + int write(byte[] buffer, int offset, int length) throws IOException { + verifyState(Writing); + Preconditions.checkArgument(buffer != null, "Null buffer"); + Preconditions.checkArgument(length >= 0, "length is negative"); + Preconditions.checkArgument(offset >= 0, "offset is negative"); + Preconditions.checkArgument( + !(buffer.length - offset < length), + "buffer shorter than amount of data to write"); + return 0; + } + + /** + * Flush the output. + * Only valid in the state {@code Writing} . + * In the base class, this is a no-op + * @throws IOException any IO problem. + */ + void flush() throws IOException { + verifyState(Writing); + } + + /** + * Switch to the upload state and return a stream for uploading. + * Base class calls {@link #enterState(DestState, DestState)} to + * manage the state machine. + * @return the stream + * @throws IOException trouble + */ + InputStream startUpload() throws IOException { + LOG.debug("Start datablock upload"); + enterState(Writing, Upload); + return null; + } + + /** + * Enter the closed state. + * @return true if the class was in any other state, implying that + * the subclass should do its close operations + */ + protected synchronized boolean enterClosedState() { + if (!state.equals(Closed)) { + enterState(null, Closed); + return true; + } else { + return false; + } + } + + @Override + public void close() throws IOException { + if (enterClosedState()) { + LOG.debug("Closed {}", this); + innerClose(); + } + } + + /** + * Inner close logic for subclasses to implement. + */ + protected void innerClose() throws IOException { + + } + + } + + // ==================================================================== + + /** + * Use byte arrays on the heap for storage. + */ + static class ArrayBlockFactory extends BlockFactory { + + ArrayBlockFactory(S3AFileSystem owner) { + super(owner); + } + + @Override + DataBlock create(int limit) throws IOException { + return new ByteArrayBlock(limit); + } + + } + + /** + * Stream to memory via a {@code ByteArrayOutputStream} . + * + * This was taken from {@code S3AFastOutputStream} and has the + * same problem which surfaced there: it consumes heap space + * proportional to the mismatch between writes to the stream and + * the JVM-wide upload bandwidth to the S3 endpoint. — End diff – but bounded by ...
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user thodemoor commented on a diff in the pull request:

          https://github.com/apache/hadoop/pull/130#discussion_r83384100

          — Diff: hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md —
          @@ -881,40 +881,362 @@ Seoul
          If the wrong endpoint is used, the request may fail. This may be reported as a 301/redirect error,
          or as a 400 Bad Request.

                1. S3AFastOutputStream
          • *Warning: NEW in hadoop 2.7. UNSTABLE, EXPERIMENTAL: use at own risk*
          • <property>
          • <name>fs.s3a.fast.upload</name>
          • <value>false</value>
          • <description>Upload directly from memory instead of buffering to
          • disk first. Memory usage and parallelism can be controlled as up to
          • fs.s3a.multipart.size memory is consumed for each (part)upload actively
          • uploading (fs.s3a.threads.max) or queueing (fs.s3a.max.total.tasks)</description>
          • </property>
          • <property>
          • <name>fs.s3a.fast.buffer.size</name>
          • <value>1048576</value>
          • <description>Size (in bytes) of initial memory buffer allocated for an
          • upload. No effect if fs.s3a.fast.upload is false.</description>
          • </property>
            +### <a name="s3a_fast_upload"></a>Stabilizing: S3A Fast Upload
            +
            +
            +*New in Hadoop 2.7; significantly enhanced in Hadoop 2.9*
            +
            +
            +Because of the nature of the S3 object store, data written to an S3A `OutputStream`
            +is not written incrementally —instead, by default, it is buffered to disk
            +until the stream is closed in its `close()` method.
            +
            +This can make output slow:
            +
            +* The execution time for `OutputStream.close()` is proportional to the amount of data
            +buffered and inversely proportional to the bandwidth. That is `O(data/bandwidth)`.
            +* The bandwidth is that available from the host to S3: other work in the same
            +process, server or network at the time of upload may increase the upload time,
            +hence the duration of the `close()` call.
            +* If a process uploading data fails before `OutputStream.close()` is called,
            +all data is lost.
            +* The disks hosting temporary directories defined in `fs.s3a.buffer.dir` must
            +have the capacity to store the entire buffered file.
            +
            +Put succinctly: the further the process is from the S3 endpoint, or the smaller
            +the EC-hosted VM is, the longer it will take work to complete.
            +
            +This can create problems in application code:
            +
            +* Code often assumes that the `close()` call is fast;
            + the delays can create bottlenecks in operations.
            +* Very slow uploads sometimes cause applications to time out. (generally,
            +threads blocking during the upload stop reporting progress, so trigger timeouts)
            +* Streaming very large amounts of data may consume all disk space before the upload begins.
            +
            +
            +Work to addess this began in Hadoop 2.7 with the `S3AFastOutputStream`
            +HADOOP-11183(https://issues.apache.org/jira/browse/HADOOP-11183), and
            +has continued with ` S3ABlockOutputStream`
            +HADOOP-13560(https://issues.apache.org/jira/browse/HADOOP-13560).
            +
            +
            +This adds an alternative output stream, "S3a Fast Upload" which:
            +
            +1. Always uploads large files as blocks with the size set by
            + `fs.s3a.multipart.size`. That is: the threshold at which multipart uploads
            + begin and the size of each upload are identical.
            +1. Buffers blocks to disk (default) or in on-heap or off-heap memory.
            +1. Uploads blocks in parallel in background threads.
            +1. Begins uploading blocks as soon as the buffered data exceeds this partition
            + size.
            +1. When buffering data to disk, uses the directory/directories listed in
            + `fs.s3a.buffer.dir`. The size of data which can be buffered is limited
            + to the available disk space.
            +1. Generates output statistics as metrics on the filesystem, including
            + statistics of active and pending block uploads.
            +1. Has the time to `close()` set by the amount of remaning data to upload, rather
            + than the total size of the file.
            +
            +With incremental writes of blocks, "S3A fast upload" offers an upload
            +time at least as fast as the "classic" mechanism, with significant benefits
            +on long-lived output streams, and when very large amounts of data are generated.
            +The in memory buffering mechanims may also offer speedup when running adjacent to
            +S3 endpoints, as disks are not used for intermediate data storage.
            +
            +
            +```xml
            +<property>
            + <name>fs.s3a.fast.upload</name>
            + <value>true</value>
            + <description>
            + Use the incremental block upload mechanism with
            + the buffering mechanism set in fs.s3a.fast.upload.buffer.
            + The number of threads performing uploads in the filesystem is defined
            + by fs.s3a.threads.max; the queue of waiting uploads limited by
            + fs.s3a.max.total.tasks.
            + The size of each buffer is set by fs.s3a.multipart.size.
            + </description>
            +</property>
            +
            +<property>
            + <name>fs.s3a.fast.upload.buffer</name>
            + <value>disk</value>
            + <description>
            + The buffering mechanism to use when using S3A fast upload
            + (fs.s3a.fast.upload=true). Values: disk, array, bytebuffer.
            + This configuration option has no effect if fs.s3a.fast.upload is false.
            +
            + "disk" will use the directories listed in fs.s3a.buffer.dir as
            + the location(s) to save data prior to being uploaded.
            +
            + "array" uses arrays in the JVM heap
            +
            + "bytebuffer" uses off-heap memory within the JVM.
            +
            + Both "array" and "bytebuffer" will consume memory in a single stream up to the number
            + of blocks set by:
            +
            + fs.s3a.multipart.size * fs.s3a.fast.upload.active.blocks.
            +
            + If using either of these mechanisms, keep this value low
            +
            + The total number of threads performing work across all threads is set by
            + fs.s3a.threads.max, with fs.s3a.max.total.tasks values setting the number of queued
            + work items.
            + </description>
            +</property>
            +
            +<property>
            + <name>fs.s3a.multipart.size</name>
            + <value>104857600</value>
            + <description>
            + How big (in bytes) to split upload or copy operations up into.
            + </description>
            +</property>
            +
            +<property>
            + <name>fs.s3a.fast.upload.active.blocks</name>
            + <value>8</value>
            + <description>
            + Maximum Number of blocks a single output stream can have
            + active (uploading, or queued to the central FileSystem
            + instance's pool of queued operations.
            +
            + This stops a single stream overloading the shared thread pool.
            + </description>
            +</property>
            +```
            +
            +*Notes*
            +
            +* If the amount of data written to a stream is below that set in `fs.s3a.multipart.size`,
            +the upload is performed in the `OutputStream.close()` operation —as with
            +the original output stream.
            +
            +* The published Hadoop metrics monitor include live queue length and
            +upload operation counts, so identifying when there is a backlog of work/
            +a mismatch between data generation rates and network bandwidth. Per-stream
            +statistics can also be logged by calling `toString()` on the current stream.
            +
            +* Incremental writes are not visible; the object can only be listed
            +or read when the multipart operation completes in the `close()` call, which
            +will block until the upload is completed.
            +
            +
            +#### <a name="s3a_fast_upload_disk"></a>Fast Upload with Disk Buffers `fs.s3a.fast.upload.buffer=disk`
            +
            +When `fs.s3a.fast.upload.buffer` is set to `disk`, all data is buffered
            +to local hard disks prior to upload. This minimizes the amount of memory
            +consumed, and so eliminates heap size as the limiting factor in queued uploads
            +—exactly as the original "direct to disk" buffering used when
            +`fs.s3a.fast.upload=false`.
            +
            +
            +```xml
            +<property>
            + <name>fs.s3a.fast.upload</name>
            + <value>true</value>
            +</property>
            +
            +<property>
            + <name>fs.s3a.fast.upload.buffer</name>
            + <value>disk</value>
            +</property>
            +
            +```
            +
            +
            +#### <a name="s3a_fast_upload_bytebuffer"></a>Fast Upload with ByteBuffers: `fs.s3a.fast.upload.buffer=bytebuffer`
            +
            +When `fs.s3a.fast.upload.buffer` is set to `bytebuffer`, all data is buffered
            +in "Direct" ByteBuffers prior to upload. This may be faster than buffering to disk,
            +and, if disk space is small (for example, tiny EC2 VMs), there may not
            +be much disk space to buffer with.
            +
            +The ByteBuffers are created in the memory of the JVM, but not in the Java Heap itself.
            +The amount of data which can be buffered is
            +limited by the Java runtime, the operating system, and, for YARN applications,
            +the amount of memory requested for each container.
            +
            +The slower the write bandwidth to S3, the greater the risk of running out
            +of memory.
              • End diff –

          Memory usage is bounded to ...

          Show
          githubbot ASF GitHub Bot added a comment - Github user thodemoor commented on a diff in the pull request: https://github.com/apache/hadoop/pull/130#discussion_r83384100 — Diff: hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md — @@ -881,40 +881,362 @@ Seoul If the wrong endpoint is used, the request may fail. This may be reported as a 301/redirect error, or as a 400 Bad Request. S3AFastOutputStream * Warning: NEW in hadoop 2.7. UNSTABLE, EXPERIMENTAL: use at own risk * <property> <name>fs.s3a.fast.upload</name> <value>false</value> <description>Upload directly from memory instead of buffering to disk first. Memory usage and parallelism can be controlled as up to fs.s3a.multipart.size memory is consumed for each (part)upload actively uploading (fs.s3a.threads.max) or queueing (fs.s3a.max.total.tasks)</description> </property> <property> <name>fs.s3a.fast.buffer.size</name> <value>1048576</value> <description>Size (in bytes) of initial memory buffer allocated for an upload. No effect if fs.s3a.fast.upload is false.</description> </property> +### <a name="s3a_fast_upload"></a>Stabilizing: S3A Fast Upload + + +* New in Hadoop 2.7; significantly enhanced in Hadoop 2.9 * + + +Because of the nature of the S3 object store, data written to an S3A `OutputStream` +is not written incrementally —instead, by default, it is buffered to disk +until the stream is closed in its `close()` method. + +This can make output slow: + +* The execution time for `OutputStream.close()` is proportional to the amount of data +buffered and inversely proportional to the bandwidth. That is `O(data/bandwidth)`. +* The bandwidth is that available from the host to S3: other work in the same +process, server or network at the time of upload may increase the upload time, +hence the duration of the `close()` call. +* If a process uploading data fails before `OutputStream.close()` is called, +all data is lost. +* The disks hosting temporary directories defined in `fs.s3a.buffer.dir` must +have the capacity to store the entire buffered file. + +Put succinctly: the further the process is from the S3 endpoint, or the smaller +the EC-hosted VM is, the longer it will take work to complete. + +This can create problems in application code: + +* Code often assumes that the `close()` call is fast; + the delays can create bottlenecks in operations. +* Very slow uploads sometimes cause applications to time out. (generally, +threads blocking during the upload stop reporting progress, so trigger timeouts) +* Streaming very large amounts of data may consume all disk space before the upload begins. + + +Work to addess this began in Hadoop 2.7 with the `S3AFastOutputStream` + HADOOP-11183 ( https://issues.apache.org/jira/browse/HADOOP-11183 ), and +has continued with ` S3ABlockOutputStream` + HADOOP-13560 ( https://issues.apache.org/jira/browse/HADOOP-13560 ). + + +This adds an alternative output stream, "S3a Fast Upload" which: + +1. Always uploads large files as blocks with the size set by + `fs.s3a.multipart.size`. That is: the threshold at which multipart uploads + begin and the size of each upload are identical. +1. Buffers blocks to disk (default) or in on-heap or off-heap memory. +1. Uploads blocks in parallel in background threads. +1. Begins uploading blocks as soon as the buffered data exceeds this partition + size. +1. When buffering data to disk, uses the directory/directories listed in + `fs.s3a.buffer.dir`. The size of data which can be buffered is limited + to the available disk space. +1. Generates output statistics as metrics on the filesystem, including + statistics of active and pending block uploads. +1. Has the time to `close()` set by the amount of remaning data to upload, rather + than the total size of the file. + +With incremental writes of blocks, "S3A fast upload" offers an upload +time at least as fast as the "classic" mechanism, with significant benefits +on long-lived output streams, and when very large amounts of data are generated. +The in memory buffering mechanims may also offer speedup when running adjacent to +S3 endpoints, as disks are not used for intermediate data storage. + + +```xml +<property> + <name>fs.s3a.fast.upload</name> + <value>true</value> + <description> + Use the incremental block upload mechanism with + the buffering mechanism set in fs.s3a.fast.upload.buffer. + The number of threads performing uploads in the filesystem is defined + by fs.s3a.threads.max; the queue of waiting uploads limited by + fs.s3a.max.total.tasks. + The size of each buffer is set by fs.s3a.multipart.size. + </description> +</property> + +<property> + <name>fs.s3a.fast.upload.buffer</name> + <value>disk</value> + <description> + The buffering mechanism to use when using S3A fast upload + (fs.s3a.fast.upload=true). Values: disk, array, bytebuffer. + This configuration option has no effect if fs.s3a.fast.upload is false. + + "disk" will use the directories listed in fs.s3a.buffer.dir as + the location(s) to save data prior to being uploaded. + + "array" uses arrays in the JVM heap + + "bytebuffer" uses off-heap memory within the JVM. + + Both "array" and "bytebuffer" will consume memory in a single stream up to the number + of blocks set by: + + fs.s3a.multipart.size * fs.s3a.fast.upload.active.blocks. + + If using either of these mechanisms, keep this value low + + The total number of threads performing work across all threads is set by + fs.s3a.threads.max, with fs.s3a.max.total.tasks values setting the number of queued + work items. + </description> +</property> + +<property> + <name>fs.s3a.multipart.size</name> + <value>104857600</value> + <description> + How big (in bytes) to split upload or copy operations up into. + </description> +</property> + +<property> + <name>fs.s3a.fast.upload.active.blocks</name> + <value>8</value> + <description> + Maximum Number of blocks a single output stream can have + active (uploading, or queued to the central FileSystem + instance's pool of queued operations. + + This stops a single stream overloading the shared thread pool. + </description> +</property> +``` + +* Notes * + +* If the amount of data written to a stream is below that set in `fs.s3a.multipart.size`, +the upload is performed in the `OutputStream.close()` operation —as with +the original output stream. + +* The published Hadoop metrics monitor include live queue length and +upload operation counts, so identifying when there is a backlog of work/ +a mismatch between data generation rates and network bandwidth. Per-stream +statistics can also be logged by calling `toString()` on the current stream. + +* Incremental writes are not visible; the object can only be listed +or read when the multipart operation completes in the `close()` call, which +will block until the upload is completed. + + +#### <a name="s3a_fast_upload_disk"></a>Fast Upload with Disk Buffers `fs.s3a.fast.upload.buffer=disk` + +When `fs.s3a.fast.upload.buffer` is set to `disk`, all data is buffered +to local hard disks prior to upload. This minimizes the amount of memory +consumed, and so eliminates heap size as the limiting factor in queued uploads +—exactly as the original "direct to disk" buffering used when +`fs.s3a.fast.upload=false`. + + +```xml +<property> + <name>fs.s3a.fast.upload</name> + <value>true</value> +</property> + +<property> + <name>fs.s3a.fast.upload.buffer</name> + <value>disk</value> +</property> + +``` + + +#### <a name="s3a_fast_upload_bytebuffer"></a>Fast Upload with ByteBuffers: `fs.s3a.fast.upload.buffer=bytebuffer` + +When `fs.s3a.fast.upload.buffer` is set to `bytebuffer`, all data is buffered +in "Direct" ByteBuffers prior to upload. This may be faster than buffering to disk, +and, if disk space is small (for example, tiny EC2 VMs), there may not +be much disk space to buffer with. + +The ByteBuffers are created in the memory of the JVM, but not in the Java Heap itself. +The amount of data which can be buffered is +limited by the Java runtime, the operating system, and, for YARN applications, +the amount of memory requested for each container. + +The slower the write bandwidth to S3, the greater the risk of running out +of memory. End diff – Memory usage is bounded to ...
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user thodemoor commented on a diff in the pull request:

          https://github.com/apache/hadoop/pull/130#discussion_r83379622

          — Diff: hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java —
          @@ -0,0 +1,699 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +
          +package org.apache.hadoop.fs.s3a;
          +
          +import java.io.IOException;
          +import java.io.InputStream;
          +import java.io.OutputStream;
          +import java.util.ArrayList;
          +import java.util.List;
          +import java.util.concurrent.Callable;
          +import java.util.concurrent.ExecutionException;
          +import java.util.concurrent.ExecutorService;
          +import java.util.concurrent.TimeUnit;
          +import java.util.concurrent.atomic.AtomicBoolean;
          +
          +import com.amazonaws.AmazonClientException;
          +import com.amazonaws.event.ProgressEvent;
          +import com.amazonaws.event.ProgressEventType;
          +import com.amazonaws.event.ProgressListener;
          +import com.amazonaws.services.s3.model.CompleteMultipartUploadResult;
          +import com.amazonaws.services.s3.model.PartETag;
          +import com.amazonaws.services.s3.model.PutObjectRequest;
          +import com.amazonaws.services.s3.model.PutObjectResult;
          +import com.amazonaws.services.s3.model.UploadPartRequest;
          +import com.google.common.base.Preconditions;
          +import com.google.common.util.concurrent.Futures;
          +import com.google.common.util.concurrent.ListenableFuture;
          +import com.google.common.util.concurrent.ListeningExecutorService;
          +import com.google.common.util.concurrent.MoreExecutors;
          +import org.slf4j.Logger;
          +import org.slf4j.LoggerFactory;
          +
          +import org.apache.hadoop.classification.InterfaceAudience;
          +import org.apache.hadoop.classification.InterfaceStability;
          +import org.apache.hadoop.io.IOUtils;
          +import org.apache.hadoop.io.retry.RetryPolicies;
          +import org.apache.hadoop.io.retry.RetryPolicy;
          +import org.apache.hadoop.util.Progressable;
          +
          +import static org.apache.hadoop.fs.s3a.S3AUtils.*;
          +import static org.apache.hadoop.fs.s3a.Statistic.*;
          +
          +/**
          + * Upload files/parts directly via different buffering mechanisms:
          + * including memory and disk.
          + *
          + * If the stream is closed and no update has started, then the upload
          + * is instead done as a single PUT operation.
          + *
          + * Unstable: statistics and error handling might evolve.
          + */
          +@InterfaceAudience.Private
          +@InterfaceStability.Unstable
          +class S3ABlockOutputStream extends OutputStream {
          +
          + private static final Logger LOG =
          + LoggerFactory.getLogger(S3ABlockOutputStream.class);
          +
          + /** Owner FileSystem. */
          + private final S3AFileSystem fs;
          +
          + /** Object being uploaded. */
          + private final String key;
          +
          + /** Size of all blocks. */
          + private final int blockSize;
          +
          + /** Callback for progress. */
          + private final ProgressListener progressListener;
          + private final ListeningExecutorService executorService;
          +
          + /**
          + * Retry policy for multipart commits; not all AWS SDK versions retry that.
          + */
          + private final RetryPolicy retryPolicy =
          + RetryPolicies.retryUpToMaximumCountWithProportionalSleep(
          + 5,
          + 2000,
          + TimeUnit.MILLISECONDS);
          + /**
          + * Factory for blocks.
          + */
          + private final S3ADataBlocks.BlockFactory blockFactory;
          +
          + /** Preallocated byte buffer for writing single characters. */
          + private final byte[] singleCharWrite = new byte[1];
          +
          + /** Multipart upload details; null means none started. */
          + private MultiPartUpload multiPartUpload;
          +
          + /** Closed flag. */
          + private final AtomicBoolean closed = new AtomicBoolean(false);
          +
          + /** Current data block. Null means none currently active */
          + private S3ADataBlocks.DataBlock activeBlock;
          +
          + /** Count of blocks uploaded. */
          + private long blockCount = 0;
          +
          + /** Statistics to build up. */
          + private final S3AInstrumentation.OutputStreamStatistics statistics;
          +
          + /**
          + * Write operation helper; encapsulation of the filesystem operations.
          + */
          + private final S3AFileSystem.WriteOperationHelper writeOperationHelper;
          +
          + /**
          + * An S3A output stream which uploads partitions in a separate pool of
          + * threads; different

          {@link S3ADataBlocks.BlockFactory}

          + * instances can control where data is buffered.
          + *
          + * @param fs S3AFilesystem
          + * @param key S3 object to work on.
          + * @param executorService the executor service to use to schedule work
          + * @param progress report progress in order to prevent timeouts. If
          + * this object implements

          {@code ProgressListener}

          then it will be
          + * directly wired up to the AWS client, so receive detailed progress
          + * information.
          + * @param blockSize size of a single block.
          + * @param blockFactory factory for creating stream destinations
          + * @param statistics stats for this stream
          + * @param writeOperationHelper state of the write operation.
          + * @throws IOException on any problem
          + */
          + S3ABlockOutputStream(S3AFileSystem fs,
          + String key,
          + ExecutorService executorService,
          + Progressable progress,
          + long blockSize,
          + S3ADataBlocks.BlockFactory blockFactory,
          + S3AInstrumentation.OutputStreamStatistics statistics,
          + S3AFileSystem.WriteOperationHelper writeOperationHelper)
          + throws IOException {
          + this.fs = fs;
          + this.key = key;
          + this.blockFactory = blockFactory;
          + this.blockSize = (int) blockSize;
          + this.statistics = statistics;
          + this.writeOperationHelper = writeOperationHelper;
          + Preconditions.checkArgument(blockSize >= Constants.MULTIPART_MIN_SIZE,
          + "Block size is too small: %d", blockSize);
          + this.executorService = MoreExecutors.listeningDecorator(executorService);
          + this.multiPartUpload = null;
          + this.progressListener = (progress instanceof ProgressListener) ?
          + (ProgressListener) progress
          + : new ProgressableListener(progress);
          + // create that first block. This guarantees that an open + close sequence
          + // writes a 0-byte entry.
          + createBlockIfNeeded();
          + LOG.debug("Initialized S3ABlockOutputStream for {}" +
          + " output to {}", writeOperationHelper, activeBlock);
          + }
          +
          + /**
          + * Demand create a destination block.
          + * @return the active block; null if there isn't one.
          + * @throws IOException on any failure to create
          + */
          + private synchronized S3ADataBlocks.DataBlock createBlockIfNeeded()
          + throws IOException {
          + if (activeBlock == null)

          { + blockCount++; + activeBlock = blockFactory.create(this.blockSize); + }

          + return activeBlock;
          + }
          +
          + /**
          + * Synchronized accessor to the active block.
          + * @return the active block; null if there isn't one.
          + */
          + private synchronized S3ADataBlocks.DataBlock getActiveBlock()

          { + return activeBlock; + }

          +
          + /**
          + * Predicate to query whether or not there is an active block.
          + * @return true if there is an active block.
          + */
          + private synchronized boolean hasActiveBlock()

          { + return activeBlock != null; + }

          +
          + /**
          + * Clear the active block.
          + */
          + private void clearActiveBlock() {
          + LOG.debug("Clearing active block");
          + synchronized (this)

          { + activeBlock = null; + }

          + }
          +
          + /**
          + * Check for the filesystem being open.
          + * @throws IOException if the filesystem is closed.
          + */
          + void checkOpen() throws IOException {
          + if (closed.get())

          { + throw new IOException("Filesystem " + writeOperationHelper + " closed"); + }

          + }
          +
          + /**
          + * The flush operation does not trigger an upload; that awaits
          + * the next block being full. What it does do is call

          {@code flush() }

          + * on the current block, leaving it to choose how to react.
          + * @throws IOException Any IO problem.
          + */
          + @Override
          + public synchronized void flush() throws IOException {
          + checkOpen();
          + S3ADataBlocks.DataBlock dataBlock = getActiveBlock();
          + if (dataBlock != null)

          { + dataBlock.flush(); + }

          + }
          +
          + /**
          + * Writes a byte to the destination. If this causes the buffer to reach
          + * its limit, the actual upload is submitted to the threadpool.
          + * @param b the int of which the lowest byte is written
          + * @throws IOException on any problem
          + */
          + @Override
          + public synchronized void write(int b) throws IOException

          { + singleCharWrite[0] = (byte)b; + write(singleCharWrite, 0, 1); + }

          +
          + /**
          + * Writes a range of bytes from to the memory buffer. If this causes the
          + * buffer to reach its limit, the actual upload is submitted to the
          + * threadpool and the remainder of the array is written to memory
          + * (recursively).
          + * @param source byte array containing
          + * @param offset offset in array where to start
          + * @param len number of bytes to be written
          + * @throws IOException on any problem
          + */
          + @Override
          + public synchronized void write(byte[] source, int offset, int len)
          + throws IOException {
          +
          + S3ADataBlocks.validateWriteArgs(source, offset, len);
          + checkOpen();
          + if (len == 0)

          { + return; + }

          + S3ADataBlocks.DataBlock block = createBlockIfNeeded();
          + int written = block.write(source, offset, len);
          + int remainingCapacity = block.remainingCapacity();
          + if (written < len) {
          + // not everything was written —the block has run out
          + // of capacity
          + // Trigger an upload then process the remainder.
          + LOG.debug("writing more data than block has capacity -triggering upload");
          + uploadCurrentBlock();
          + // tail recursion is mildly expensive, but given buffer sizes must be MB.
          — End diff –

          FYI Up to 10k. That's AWS's limit on the number of parts in a single multipartupload.

          Show
          githubbot ASF GitHub Bot added a comment - Github user thodemoor commented on a diff in the pull request: https://github.com/apache/hadoop/pull/130#discussion_r83379622 — Diff: hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java — @@ -0,0 +1,699 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.s3a; + +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.Callable; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; + +import com.amazonaws.AmazonClientException; +import com.amazonaws.event.ProgressEvent; +import com.amazonaws.event.ProgressEventType; +import com.amazonaws.event.ProgressListener; +import com.amazonaws.services.s3.model.CompleteMultipartUploadResult; +import com.amazonaws.services.s3.model.PartETag; +import com.amazonaws.services.s3.model.PutObjectRequest; +import com.amazonaws.services.s3.model.PutObjectResult; +import com.amazonaws.services.s3.model.UploadPartRequest; +import com.google.common.base.Preconditions; +import com.google.common.util.concurrent.Futures; +import com.google.common.util.concurrent.ListenableFuture; +import com.google.common.util.concurrent.ListeningExecutorService; +import com.google.common.util.concurrent.MoreExecutors; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.io.IOUtils; +import org.apache.hadoop.io.retry.RetryPolicies; +import org.apache.hadoop.io.retry.RetryPolicy; +import org.apache.hadoop.util.Progressable; + +import static org.apache.hadoop.fs.s3a.S3AUtils.*; +import static org.apache.hadoop.fs.s3a.Statistic.*; + +/** + * Upload files/parts directly via different buffering mechanisms: + * including memory and disk. + * + * If the stream is closed and no update has started, then the upload + * is instead done as a single PUT operation. + * + * Unstable: statistics and error handling might evolve. + */ +@InterfaceAudience.Private +@InterfaceStability.Unstable +class S3ABlockOutputStream extends OutputStream { + + private static final Logger LOG = + LoggerFactory.getLogger(S3ABlockOutputStream.class); + + /** Owner FileSystem. */ + private final S3AFileSystem fs; + + /** Object being uploaded. */ + private final String key; + + /** Size of all blocks. */ + private final int blockSize; + + /** Callback for progress. */ + private final ProgressListener progressListener; + private final ListeningExecutorService executorService; + + /** + * Retry policy for multipart commits; not all AWS SDK versions retry that. + */ + private final RetryPolicy retryPolicy = + RetryPolicies.retryUpToMaximumCountWithProportionalSleep( + 5, + 2000, + TimeUnit.MILLISECONDS); + /** + * Factory for blocks. + */ + private final S3ADataBlocks.BlockFactory blockFactory; + + /** Preallocated byte buffer for writing single characters. */ + private final byte[] singleCharWrite = new byte [1] ; + + /** Multipart upload details; null means none started. */ + private MultiPartUpload multiPartUpload; + + /** Closed flag. */ + private final AtomicBoolean closed = new AtomicBoolean(false); + + /** Current data block. Null means none currently active */ + private S3ADataBlocks.DataBlock activeBlock; + + /** Count of blocks uploaded. */ + private long blockCount = 0; + + /** Statistics to build up. */ + private final S3AInstrumentation.OutputStreamStatistics statistics; + + /** + * Write operation helper; encapsulation of the filesystem operations. + */ + private final S3AFileSystem.WriteOperationHelper writeOperationHelper; + + /** + * An S3A output stream which uploads partitions in a separate pool of + * threads; different {@link S3ADataBlocks.BlockFactory} + * instances can control where data is buffered. + * + * @param fs S3AFilesystem + * @param key S3 object to work on. + * @param executorService the executor service to use to schedule work + * @param progress report progress in order to prevent timeouts. If + * this object implements {@code ProgressListener} then it will be + * directly wired up to the AWS client, so receive detailed progress + * information. + * @param blockSize size of a single block. + * @param blockFactory factory for creating stream destinations + * @param statistics stats for this stream + * @param writeOperationHelper state of the write operation. + * @throws IOException on any problem + */ + S3ABlockOutputStream(S3AFileSystem fs, + String key, + ExecutorService executorService, + Progressable progress, + long blockSize, + S3ADataBlocks.BlockFactory blockFactory, + S3AInstrumentation.OutputStreamStatistics statistics, + S3AFileSystem.WriteOperationHelper writeOperationHelper) + throws IOException { + this.fs = fs; + this.key = key; + this.blockFactory = blockFactory; + this.blockSize = (int) blockSize; + this.statistics = statistics; + this.writeOperationHelper = writeOperationHelper; + Preconditions.checkArgument(blockSize >= Constants.MULTIPART_MIN_SIZE, + "Block size is too small: %d", blockSize); + this.executorService = MoreExecutors.listeningDecorator(executorService); + this.multiPartUpload = null; + this.progressListener = (progress instanceof ProgressListener) ? + (ProgressListener) progress + : new ProgressableListener(progress); + // create that first block. This guarantees that an open + close sequence + // writes a 0-byte entry. + createBlockIfNeeded(); + LOG.debug("Initialized S3ABlockOutputStream for {}" + + " output to {}", writeOperationHelper, activeBlock); + } + + /** + * Demand create a destination block. + * @return the active block; null if there isn't one. + * @throws IOException on any failure to create + */ + private synchronized S3ADataBlocks.DataBlock createBlockIfNeeded() + throws IOException { + if (activeBlock == null) { + blockCount++; + activeBlock = blockFactory.create(this.blockSize); + } + return activeBlock; + } + + /** + * Synchronized accessor to the active block. + * @return the active block; null if there isn't one. + */ + private synchronized S3ADataBlocks.DataBlock getActiveBlock() { + return activeBlock; + } + + /** + * Predicate to query whether or not there is an active block. + * @return true if there is an active block. + */ + private synchronized boolean hasActiveBlock() { + return activeBlock != null; + } + + /** + * Clear the active block. + */ + private void clearActiveBlock() { + LOG.debug("Clearing active block"); + synchronized (this) { + activeBlock = null; + } + } + + /** + * Check for the filesystem being open. + * @throws IOException if the filesystem is closed. + */ + void checkOpen() throws IOException { + if (closed.get()) { + throw new IOException("Filesystem " + writeOperationHelper + " closed"); + } + } + + /** + * The flush operation does not trigger an upload; that awaits + * the next block being full. What it does do is call {@code flush() } + * on the current block, leaving it to choose how to react. + * @throws IOException Any IO problem. + */ + @Override + public synchronized void flush() throws IOException { + checkOpen(); + S3ADataBlocks.DataBlock dataBlock = getActiveBlock(); + if (dataBlock != null) { + dataBlock.flush(); + } + } + + /** + * Writes a byte to the destination. If this causes the buffer to reach + * its limit, the actual upload is submitted to the threadpool. + * @param b the int of which the lowest byte is written + * @throws IOException on any problem + */ + @Override + public synchronized void write(int b) throws IOException { + singleCharWrite[0] = (byte)b; + write(singleCharWrite, 0, 1); + } + + /** + * Writes a range of bytes from to the memory buffer. If this causes the + * buffer to reach its limit, the actual upload is submitted to the + * threadpool and the remainder of the array is written to memory + * (recursively). + * @param source byte array containing + * @param offset offset in array where to start + * @param len number of bytes to be written + * @throws IOException on any problem + */ + @Override + public synchronized void write(byte[] source, int offset, int len) + throws IOException { + + S3ADataBlocks.validateWriteArgs(source, offset, len); + checkOpen(); + if (len == 0) { + return; + } + S3ADataBlocks.DataBlock block = createBlockIfNeeded(); + int written = block.write(source, offset, len); + int remainingCapacity = block.remainingCapacity(); + if (written < len) { + // not everything was written —the block has run out + // of capacity + // Trigger an upload then process the remainder. + LOG.debug("writing more data than block has capacity -triggering upload"); + uploadCurrentBlock(); + // tail recursion is mildly expensive, but given buffer sizes must be MB. — End diff – FYI Up to 10k. That's AWS's limit on the number of parts in a single multipartupload.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user thodemoor commented on a diff in the pull request:

          https://github.com/apache/hadoop/pull/130#discussion_r83384371

          — Diff: hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md —
          @@ -881,40 +881,362 @@ Seoul
          If the wrong endpoint is used, the request may fail. This may be reported as a 301/redirect error,
          or as a 400 Bad Request.

                1. S3AFastOutputStream
          • *Warning: NEW in hadoop 2.7. UNSTABLE, EXPERIMENTAL: use at own risk*
          • <property>
          • <name>fs.s3a.fast.upload</name>
          • <value>false</value>
          • <description>Upload directly from memory instead of buffering to
          • disk first. Memory usage and parallelism can be controlled as up to
          • fs.s3a.multipart.size memory is consumed for each (part)upload actively
          • uploading (fs.s3a.threads.max) or queueing (fs.s3a.max.total.tasks)</description>
          • </property>
          • <property>
          • <name>fs.s3a.fast.buffer.size</name>
          • <value>1048576</value>
          • <description>Size (in bytes) of initial memory buffer allocated for an
          • upload. No effect if fs.s3a.fast.upload is false.</description>
          • </property>
            +### <a name="s3a_fast_upload"></a>Stabilizing: S3A Fast Upload
            +
            +
            +*New in Hadoop 2.7; significantly enhanced in Hadoop 2.9*
            +
            +
            +Because of the nature of the S3 object store, data written to an S3A `OutputStream`
            +is not written incrementally —instead, by default, it is buffered to disk
            +until the stream is closed in its `close()` method.
            +
            +This can make output slow:
            +
            +* The execution time for `OutputStream.close()` is proportional to the amount of data
            +buffered and inversely proportional to the bandwidth. That is `O(data/bandwidth)`.
            +* The bandwidth is that available from the host to S3: other work in the same
            +process, server or network at the time of upload may increase the upload time,
            +hence the duration of the `close()` call.
            +* If a process uploading data fails before `OutputStream.close()` is called,
            +all data is lost.
            +* The disks hosting temporary directories defined in `fs.s3a.buffer.dir` must
            +have the capacity to store the entire buffered file.
            +
            +Put succinctly: the further the process is from the S3 endpoint, or the smaller
            +the EC-hosted VM is, the longer it will take work to complete.
            +
            +This can create problems in application code:
            +
            +* Code often assumes that the `close()` call is fast;
            + the delays can create bottlenecks in operations.
            +* Very slow uploads sometimes cause applications to time out. (generally,
            +threads blocking during the upload stop reporting progress, so trigger timeouts)
            +* Streaming very large amounts of data may consume all disk space before the upload begins.
            +
            +
            +Work to addess this began in Hadoop 2.7 with the `S3AFastOutputStream`
            +HADOOP-11183(https://issues.apache.org/jira/browse/HADOOP-11183), and
            +has continued with ` S3ABlockOutputStream`
            +HADOOP-13560(https://issues.apache.org/jira/browse/HADOOP-13560).
            +
            +
            +This adds an alternative output stream, "S3a Fast Upload" which:
            +
            +1. Always uploads large files as blocks with the size set by
            + `fs.s3a.multipart.size`. That is: the threshold at which multipart uploads
            + begin and the size of each upload are identical.
            +1. Buffers blocks to disk (default) or in on-heap or off-heap memory.
            +1. Uploads blocks in parallel in background threads.
            +1. Begins uploading blocks as soon as the buffered data exceeds this partition
            + size.
            +1. When buffering data to disk, uses the directory/directories listed in
            + `fs.s3a.buffer.dir`. The size of data which can be buffered is limited
            + to the available disk space.
            +1. Generates output statistics as metrics on the filesystem, including
            + statistics of active and pending block uploads.
            +1. Has the time to `close()` set by the amount of remaning data to upload, rather
            + than the total size of the file.
            +
            +With incremental writes of blocks, "S3A fast upload" offers an upload
            +time at least as fast as the "classic" mechanism, with significant benefits
            +on long-lived output streams, and when very large amounts of data are generated.
            +The in memory buffering mechanims may also offer speedup when running adjacent to
            +S3 endpoints, as disks are not used for intermediate data storage.
            +
            +
            +```xml
            +<property>
            + <name>fs.s3a.fast.upload</name>
            + <value>true</value>
            + <description>
            + Use the incremental block upload mechanism with
            + the buffering mechanism set in fs.s3a.fast.upload.buffer.
            + The number of threads performing uploads in the filesystem is defined
            + by fs.s3a.threads.max; the queue of waiting uploads limited by
            + fs.s3a.max.total.tasks.
            + The size of each buffer is set by fs.s3a.multipart.size.
            + </description>
            +</property>
            +
            +<property>
            + <name>fs.s3a.fast.upload.buffer</name>
            + <value>disk</value>
            + <description>
            + The buffering mechanism to use when using S3A fast upload
            + (fs.s3a.fast.upload=true). Values: disk, array, bytebuffer.
            + This configuration option has no effect if fs.s3a.fast.upload is false.
            +
            + "disk" will use the directories listed in fs.s3a.buffer.dir as
            + the location(s) to save data prior to being uploaded.
            +
            + "array" uses arrays in the JVM heap
            +
            + "bytebuffer" uses off-heap memory within the JVM.
            +
            + Both "array" and "bytebuffer" will consume memory in a single stream up to the number
            + of blocks set by:
            +
            + fs.s3a.multipart.size * fs.s3a.fast.upload.active.blocks.
            +
            + If using either of these mechanisms, keep this value low
            +
            + The total number of threads performing work across all threads is set by
            + fs.s3a.threads.max, with fs.s3a.max.total.tasks values setting the number of queued
            + work items.
            + </description>
            +</property>
            +
            +<property>
            + <name>fs.s3a.multipart.size</name>
            + <value>104857600</value>
            + <description>
            + How big (in bytes) to split upload or copy operations up into.
            + </description>
            +</property>
            +
            +<property>
            + <name>fs.s3a.fast.upload.active.blocks</name>
            + <value>8</value>
            + <description>
            + Maximum Number of blocks a single output stream can have
            + active (uploading, or queued to the central FileSystem
            + instance's pool of queued operations.
            +
            + This stops a single stream overloading the shared thread pool.
            + </description>
            +</property>
            +```
            +
            +*Notes*
            +
            +* If the amount of data written to a stream is below that set in `fs.s3a.multipart.size`,
            +the upload is performed in the `OutputStream.close()` operation —as with
            +the original output stream.
            +
            +* The published Hadoop metrics monitor include live queue length and
            +upload operation counts, so identifying when there is a backlog of work/
            +a mismatch between data generation rates and network bandwidth. Per-stream
            +statistics can also be logged by calling `toString()` on the current stream.
            +
            +* Incremental writes are not visible; the object can only be listed
            +or read when the multipart operation completes in the `close()` call, which
            +will block until the upload is completed.
            +
            +
            +#### <a name="s3a_fast_upload_disk"></a>Fast Upload with Disk Buffers `fs.s3a.fast.upload.buffer=disk`
            +
            +When `fs.s3a.fast.upload.buffer` is set to `disk`, all data is buffered
            +to local hard disks prior to upload. This minimizes the amount of memory
            +consumed, and so eliminates heap size as the limiting factor in queued uploads
            +—exactly as the original "direct to disk" buffering used when
            +`fs.s3a.fast.upload=false`.
            +
            +
            +```xml
            +<property>
            + <name>fs.s3a.fast.upload</name>
            + <value>true</value>
            +</property>
            +
            +<property>
            + <name>fs.s3a.fast.upload.buffer</name>
            + <value>disk</value>
            +</property>
            +
            +```
            +
            +
            +#### <a name="s3a_fast_upload_bytebuffer"></a>Fast Upload with ByteBuffers: `fs.s3a.fast.upload.buffer=bytebuffer`
            +
            +When `fs.s3a.fast.upload.buffer` is set to `bytebuffer`, all data is buffered
            +in "Direct" ByteBuffers prior to upload. This may be faster than buffering to disk,
            +and, if disk space is small (for example, tiny EC2 VMs), there may not
            +be much disk space to buffer with.
            +
            +The ByteBuffers are created in the memory of the JVM, but not in the Java Heap itself.
            +The amount of data which can be buffered is
            +limited by the Java runtime, the operating system, and, for YARN applications,
            +the amount of memory requested for each container.
            +
            +The slower the write bandwidth to S3, the greater the risk of running out
            +of memory.
            +
            +
            +```xml
            +<property>
            + <name>fs.s3a.fast.upload</name>
            + <value>true</value>
            +</property>
            +
            +<property>
            + <name>fs.s3a.fast.upload.buffer</name>
            + <value>bytebuffer</value>
            +</property>
            +```
            +
            +#### <a name="s3a_fast_upload_array"></a>Fast Upload with Arrays: `fs.s3a.fast.upload.buffer=array`
            +
            +When `fs.s3a.fast.upload.buffer` is set to `array`, all data is buffered
            +in byte arrays in the JVM's heap prior to upload.
            +This may be faster than buffering to disk.
            +
            +This `array` option is similar to the in-memory-only stream offered in
            +Hadoop 2.7 with `fs.s3a.fast.upload=true`
            +
            +The amount of data which can be buffered is limited by the available
            +size of the JVM heap heap. The slower the write bandwidth to S3, the greater
            +the risk of heap overflows.
              • End diff –

          idem

          Show
          githubbot ASF GitHub Bot added a comment - Github user thodemoor commented on a diff in the pull request: https://github.com/apache/hadoop/pull/130#discussion_r83384371 — Diff: hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md — @@ -881,40 +881,362 @@ Seoul If the wrong endpoint is used, the request may fail. This may be reported as a 301/redirect error, or as a 400 Bad Request. S3AFastOutputStream * Warning: NEW in hadoop 2.7. UNSTABLE, EXPERIMENTAL: use at own risk * <property> <name>fs.s3a.fast.upload</name> <value>false</value> <description>Upload directly from memory instead of buffering to disk first. Memory usage and parallelism can be controlled as up to fs.s3a.multipart.size memory is consumed for each (part)upload actively uploading (fs.s3a.threads.max) or queueing (fs.s3a.max.total.tasks)</description> </property> <property> <name>fs.s3a.fast.buffer.size</name> <value>1048576</value> <description>Size (in bytes) of initial memory buffer allocated for an upload. No effect if fs.s3a.fast.upload is false.</description> </property> +### <a name="s3a_fast_upload"></a>Stabilizing: S3A Fast Upload + + +* New in Hadoop 2.7; significantly enhanced in Hadoop 2.9 * + + +Because of the nature of the S3 object store, data written to an S3A `OutputStream` +is not written incrementally —instead, by default, it is buffered to disk +until the stream is closed in its `close()` method. + +This can make output slow: + +* The execution time for `OutputStream.close()` is proportional to the amount of data +buffered and inversely proportional to the bandwidth. That is `O(data/bandwidth)`. +* The bandwidth is that available from the host to S3: other work in the same +process, server or network at the time of upload may increase the upload time, +hence the duration of the `close()` call. +* If a process uploading data fails before `OutputStream.close()` is called, +all data is lost. +* The disks hosting temporary directories defined in `fs.s3a.buffer.dir` must +have the capacity to store the entire buffered file. + +Put succinctly: the further the process is from the S3 endpoint, or the smaller +the EC-hosted VM is, the longer it will take work to complete. + +This can create problems in application code: + +* Code often assumes that the `close()` call is fast; + the delays can create bottlenecks in operations. +* Very slow uploads sometimes cause applications to time out. (generally, +threads blocking during the upload stop reporting progress, so trigger timeouts) +* Streaming very large amounts of data may consume all disk space before the upload begins. + + +Work to addess this began in Hadoop 2.7 with the `S3AFastOutputStream` + HADOOP-11183 ( https://issues.apache.org/jira/browse/HADOOP-11183 ), and +has continued with ` S3ABlockOutputStream` + HADOOP-13560 ( https://issues.apache.org/jira/browse/HADOOP-13560 ). + + +This adds an alternative output stream, "S3a Fast Upload" which: + +1. Always uploads large files as blocks with the size set by + `fs.s3a.multipart.size`. That is: the threshold at which multipart uploads + begin and the size of each upload are identical. +1. Buffers blocks to disk (default) or in on-heap or off-heap memory. +1. Uploads blocks in parallel in background threads. +1. Begins uploading blocks as soon as the buffered data exceeds this partition + size. +1. When buffering data to disk, uses the directory/directories listed in + `fs.s3a.buffer.dir`. The size of data which can be buffered is limited + to the available disk space. +1. Generates output statistics as metrics on the filesystem, including + statistics of active and pending block uploads. +1. Has the time to `close()` set by the amount of remaning data to upload, rather + than the total size of the file. + +With incremental writes of blocks, "S3A fast upload" offers an upload +time at least as fast as the "classic" mechanism, with significant benefits +on long-lived output streams, and when very large amounts of data are generated. +The in memory buffering mechanims may also offer speedup when running adjacent to +S3 endpoints, as disks are not used for intermediate data storage. + + +```xml +<property> + <name>fs.s3a.fast.upload</name> + <value>true</value> + <description> + Use the incremental block upload mechanism with + the buffering mechanism set in fs.s3a.fast.upload.buffer. + The number of threads performing uploads in the filesystem is defined + by fs.s3a.threads.max; the queue of waiting uploads limited by + fs.s3a.max.total.tasks. + The size of each buffer is set by fs.s3a.multipart.size. + </description> +</property> + +<property> + <name>fs.s3a.fast.upload.buffer</name> + <value>disk</value> + <description> + The buffering mechanism to use when using S3A fast upload + (fs.s3a.fast.upload=true). Values: disk, array, bytebuffer. + This configuration option has no effect if fs.s3a.fast.upload is false. + + "disk" will use the directories listed in fs.s3a.buffer.dir as + the location(s) to save data prior to being uploaded. + + "array" uses arrays in the JVM heap + + "bytebuffer" uses off-heap memory within the JVM. + + Both "array" and "bytebuffer" will consume memory in a single stream up to the number + of blocks set by: + + fs.s3a.multipart.size * fs.s3a.fast.upload.active.blocks. + + If using either of these mechanisms, keep this value low + + The total number of threads performing work across all threads is set by + fs.s3a.threads.max, with fs.s3a.max.total.tasks values setting the number of queued + work items. + </description> +</property> + +<property> + <name>fs.s3a.multipart.size</name> + <value>104857600</value> + <description> + How big (in bytes) to split upload or copy operations up into. + </description> +</property> + +<property> + <name>fs.s3a.fast.upload.active.blocks</name> + <value>8</value> + <description> + Maximum Number of blocks a single output stream can have + active (uploading, or queued to the central FileSystem + instance's pool of queued operations. + + This stops a single stream overloading the shared thread pool. + </description> +</property> +``` + +* Notes * + +* If the amount of data written to a stream is below that set in `fs.s3a.multipart.size`, +the upload is performed in the `OutputStream.close()` operation —as with +the original output stream. + +* The published Hadoop metrics monitor include live queue length and +upload operation counts, so identifying when there is a backlog of work/ +a mismatch between data generation rates and network bandwidth. Per-stream +statistics can also be logged by calling `toString()` on the current stream. + +* Incremental writes are not visible; the object can only be listed +or read when the multipart operation completes in the `close()` call, which +will block until the upload is completed. + + +#### <a name="s3a_fast_upload_disk"></a>Fast Upload with Disk Buffers `fs.s3a.fast.upload.buffer=disk` + +When `fs.s3a.fast.upload.buffer` is set to `disk`, all data is buffered +to local hard disks prior to upload. This minimizes the amount of memory +consumed, and so eliminates heap size as the limiting factor in queued uploads +—exactly as the original "direct to disk" buffering used when +`fs.s3a.fast.upload=false`. + + +```xml +<property> + <name>fs.s3a.fast.upload</name> + <value>true</value> +</property> + +<property> + <name>fs.s3a.fast.upload.buffer</name> + <value>disk</value> +</property> + +``` + + +#### <a name="s3a_fast_upload_bytebuffer"></a>Fast Upload with ByteBuffers: `fs.s3a.fast.upload.buffer=bytebuffer` + +When `fs.s3a.fast.upload.buffer` is set to `bytebuffer`, all data is buffered +in "Direct" ByteBuffers prior to upload. This may be faster than buffering to disk, +and, if disk space is small (for example, tiny EC2 VMs), there may not +be much disk space to buffer with. + +The ByteBuffers are created in the memory of the JVM, but not in the Java Heap itself. +The amount of data which can be buffered is +limited by the Java runtime, the operating system, and, for YARN applications, +the amount of memory requested for each container. + +The slower the write bandwidth to S3, the greater the risk of running out +of memory. + + +```xml +<property> + <name>fs.s3a.fast.upload</name> + <value>true</value> +</property> + +<property> + <name>fs.s3a.fast.upload.buffer</name> + <value>bytebuffer</value> +</property> +``` + +#### <a name="s3a_fast_upload_array"></a>Fast Upload with Arrays: `fs.s3a.fast.upload.buffer=array` + +When `fs.s3a.fast.upload.buffer` is set to `array`, all data is buffered +in byte arrays in the JVM's heap prior to upload. +This may be faster than buffering to disk. + +This `array` option is similar to the in-memory-only stream offered in +Hadoop 2.7 with `fs.s3a.fast.upload=true` + +The amount of data which can be buffered is limited by the available +size of the JVM heap heap. The slower the write bandwidth to S3, the greater +the risk of heap overflows. End diff – idem
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user thodemoor commented on a diff in the pull request:

          https://github.com/apache/hadoop/pull/130#discussion_r83384867

          — Diff: hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md —
          @@ -881,40 +881,362 @@ Seoul
          If the wrong endpoint is used, the request may fail. This may be reported as a 301/redirect error,
          or as a 400 Bad Request.

                1. S3AFastOutputStream
          • *Warning: NEW in hadoop 2.7. UNSTABLE, EXPERIMENTAL: use at own risk*
          • <property>
          • <name>fs.s3a.fast.upload</name>
          • <value>false</value>
          • <description>Upload directly from memory instead of buffering to
          • disk first. Memory usage and parallelism can be controlled as up to
          • fs.s3a.multipart.size memory is consumed for each (part)upload actively
          • uploading (fs.s3a.threads.max) or queueing (fs.s3a.max.total.tasks)</description>
          • </property>
          • <property>
          • <name>fs.s3a.fast.buffer.size</name>
          • <value>1048576</value>
          • <description>Size (in bytes) of initial memory buffer allocated for an
          • upload. No effect if fs.s3a.fast.upload is false.</description>
          • </property>
            +### <a name="s3a_fast_upload"></a>Stabilizing: S3A Fast Upload
            +
            +
            +*New in Hadoop 2.7; significantly enhanced in Hadoop 2.9*
            +
            +
            +Because of the nature of the S3 object store, data written to an S3A `OutputStream`
            +is not written incrementally —instead, by default, it is buffered to disk
            +until the stream is closed in its `close()` method.
            +
            +This can make output slow:
            +
            +* The execution time for `OutputStream.close()` is proportional to the amount of data
            +buffered and inversely proportional to the bandwidth. That is `O(data/bandwidth)`.
            +* The bandwidth is that available from the host to S3: other work in the same
            +process, server or network at the time of upload may increase the upload time,
            +hence the duration of the `close()` call.
            +* If a process uploading data fails before `OutputStream.close()` is called,
            +all data is lost.
            +* The disks hosting temporary directories defined in `fs.s3a.buffer.dir` must
            +have the capacity to store the entire buffered file.
            +
            +Put succinctly: the further the process is from the S3 endpoint, or the smaller
            +the EC-hosted VM is, the longer it will take work to complete.
            +
            +This can create problems in application code:
            +
            +* Code often assumes that the `close()` call is fast;
            + the delays can create bottlenecks in operations.
            +* Very slow uploads sometimes cause applications to time out. (generally,
            +threads blocking during the upload stop reporting progress, so trigger timeouts)
            +* Streaming very large amounts of data may consume all disk space before the upload begins.
            +
            +
            +Work to addess this began in Hadoop 2.7 with the `S3AFastOutputStream`
            +HADOOP-11183(https://issues.apache.org/jira/browse/HADOOP-11183), and
            +has continued with ` S3ABlockOutputStream`
            +HADOOP-13560(https://issues.apache.org/jira/browse/HADOOP-13560).
            +
            +
            +This adds an alternative output stream, "S3a Fast Upload" which:
            +
            +1. Always uploads large files as blocks with the size set by
            + `fs.s3a.multipart.size`. That is: the threshold at which multipart uploads
            + begin and the size of each upload are identical.
            +1. Buffers blocks to disk (default) or in on-heap or off-heap memory.
            +1. Uploads blocks in parallel in background threads.
            +1. Begins uploading blocks as soon as the buffered data exceeds this partition
            + size.
            +1. When buffering data to disk, uses the directory/directories listed in
            + `fs.s3a.buffer.dir`. The size of data which can be buffered is limited
            + to the available disk space.
            +1. Generates output statistics as metrics on the filesystem, including
            + statistics of active and pending block uploads.
            +1. Has the time to `close()` set by the amount of remaning data to upload, rather
            + than the total size of the file.
            +
            +With incremental writes of blocks, "S3A fast upload" offers an upload
            +time at least as fast as the "classic" mechanism, with significant benefits
            +on long-lived output streams, and when very large amounts of data are generated.
            +The in memory buffering mechanims may also offer speedup when running adjacent to
            +S3 endpoints, as disks are not used for intermediate data storage.
            +
            +
            +```xml
            +<property>
            + <name>fs.s3a.fast.upload</name>
            + <value>true</value>
            + <description>
            + Use the incremental block upload mechanism with
            + the buffering mechanism set in fs.s3a.fast.upload.buffer.
            + The number of threads performing uploads in the filesystem is defined
            + by fs.s3a.threads.max; the queue of waiting uploads limited by
            + fs.s3a.max.total.tasks.
            + The size of each buffer is set by fs.s3a.multipart.size.
            + </description>
            +</property>
            +
            +<property>
            + <name>fs.s3a.fast.upload.buffer</name>
            + <value>disk</value>
            + <description>
            + The buffering mechanism to use when using S3A fast upload
            + (fs.s3a.fast.upload=true). Values: disk, array, bytebuffer.
            + This configuration option has no effect if fs.s3a.fast.upload is false.
            +
            + "disk" will use the directories listed in fs.s3a.buffer.dir as
            + the location(s) to save data prior to being uploaded.
            +
            + "array" uses arrays in the JVM heap
            +
            + "bytebuffer" uses off-heap memory within the JVM.
            +
            + Both "array" and "bytebuffer" will consume memory in a single stream up to the number
            + of blocks set by:
            +
            + fs.s3a.multipart.size * fs.s3a.fast.upload.active.blocks.
            +
            + If using either of these mechanisms, keep this value low
            +
            + The total number of threads performing work across all threads is set by
            + fs.s3a.threads.max, with fs.s3a.max.total.tasks values setting the number of queued
            + work items.
            + </description>
            +</property>
            +
            +<property>
            + <name>fs.s3a.multipart.size</name>
            + <value>104857600</value>
            + <description>
            + How big (in bytes) to split upload or copy operations up into.
            + </description>
            +</property>
            +
            +<property>
            + <name>fs.s3a.fast.upload.active.blocks</name>
            + <value>8</value>
            + <description>
            + Maximum Number of blocks a single output stream can have
            + active (uploading, or queued to the central FileSystem
            + instance's pool of queued operations.
            +
            + This stops a single stream overloading the shared thread pool.
            + </description>
            +</property>
            +```
            +
            +*Notes*
            +
            +* If the amount of data written to a stream is below that set in `fs.s3a.multipart.size`,
            +the upload is performed in the `OutputStream.close()` operation —as with
            +the original output stream.
            +
            +* The published Hadoop metrics monitor include live queue length and
            +upload operation counts, so identifying when there is a backlog of work/
            +a mismatch between data generation rates and network bandwidth. Per-stream
            +statistics can also be logged by calling `toString()` on the current stream.
            +
            +* Incremental writes are not visible; the object can only be listed
            +or read when the multipart operation completes in the `close()` call, which
            +will block until the upload is completed.
            +
            +
            +#### <a name="s3a_fast_upload_disk"></a>Fast Upload with Disk Buffers `fs.s3a.fast.upload.buffer=disk`
            +
            +When `fs.s3a.fast.upload.buffer` is set to `disk`, all data is buffered
            +to local hard disks prior to upload. This minimizes the amount of memory
            +consumed, and so eliminates heap size as the limiting factor in queued uploads
            +—exactly as the original "direct to disk" buffering used when
            +`fs.s3a.fast.upload=false`.
            +
            +
            +```xml
            +<property>
            + <name>fs.s3a.fast.upload</name>
            + <value>true</value>
            +</property>
            +
            +<property>
            + <name>fs.s3a.fast.upload.buffer</name>
            + <value>disk</value>
            +</property>
            +
            +```
            +
            +
            +#### <a name="s3a_fast_upload_bytebuffer"></a>Fast Upload with ByteBuffers: `fs.s3a.fast.upload.buffer=bytebuffer`
            +
            +When `fs.s3a.fast.upload.buffer` is set to `bytebuffer`, all data is buffered
            +in "Direct" ByteBuffers prior to upload. This may be faster than buffering to disk,
            +and, if disk space is small (for example, tiny EC2 VMs), there may not
            +be much disk space to buffer with.
            +
            +The ByteBuffers are created in the memory of the JVM, but not in the Java Heap itself.
            +The amount of data which can be buffered is
            +limited by the Java runtime, the operating system, and, for YARN applications,
            +the amount of memory requested for each container.
            +
            +The slower the write bandwidth to S3, the greater the risk of running out
            +of memory.
            +
            +
            +```xml
            +<property>
            + <name>fs.s3a.fast.upload</name>
            + <value>true</value>
            +</property>
            +
            +<property>
            + <name>fs.s3a.fast.upload.buffer</name>
            + <value>bytebuffer</value>
            +</property>
            +```
            +
            +#### <a name="s3a_fast_upload_array"></a>Fast Upload with Arrays: `fs.s3a.fast.upload.buffer=array`
            +
            +When `fs.s3a.fast.upload.buffer` is set to `array`, all data is buffered
            +in byte arrays in the JVM's heap prior to upload.
            +This may be faster than buffering to disk.
            +
            +This `array` option is similar to the in-memory-only stream offered in
            +Hadoop 2.7 with `fs.s3a.fast.upload=true`
            +
            +The amount of data which can be buffered is limited by the available
            +size of the JVM heap heap. The slower the write bandwidth to S3, the greater
            +the risk of heap overflows.
            +
            +```xml
            +<property>
            + <name>fs.s3a.fast.upload</name>
            + <value>true</value>
            +</property>
            +
            +<property>
            + <name>fs.s3a.fast.upload.buffer</name>
            + <value>array</value>
            +</property>
            +
            +```
            +#### <a name="s3a_fast_upload_thread_tuning"></a>S3A Fast Upload Thread Tuning
            +
              • End diff –

          As a (probably better) alternative to my other comments, we could explain the bound on the memory consumption here once and link to it.

          Show
          githubbot ASF GitHub Bot added a comment - Github user thodemoor commented on a diff in the pull request: https://github.com/apache/hadoop/pull/130#discussion_r83384867 — Diff: hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md — @@ -881,40 +881,362 @@ Seoul If the wrong endpoint is used, the request may fail. This may be reported as a 301/redirect error, or as a 400 Bad Request. S3AFastOutputStream * Warning: NEW in hadoop 2.7. UNSTABLE, EXPERIMENTAL: use at own risk * <property> <name>fs.s3a.fast.upload</name> <value>false</value> <description>Upload directly from memory instead of buffering to disk first. Memory usage and parallelism can be controlled as up to fs.s3a.multipart.size memory is consumed for each (part)upload actively uploading (fs.s3a.threads.max) or queueing (fs.s3a.max.total.tasks)</description> </property> <property> <name>fs.s3a.fast.buffer.size</name> <value>1048576</value> <description>Size (in bytes) of initial memory buffer allocated for an upload. No effect if fs.s3a.fast.upload is false.</description> </property> +### <a name="s3a_fast_upload"></a>Stabilizing: S3A Fast Upload + + +* New in Hadoop 2.7; significantly enhanced in Hadoop 2.9 * + + +Because of the nature of the S3 object store, data written to an S3A `OutputStream` +is not written incrementally —instead, by default, it is buffered to disk +until the stream is closed in its `close()` method. + +This can make output slow: + +* The execution time for `OutputStream.close()` is proportional to the amount of data +buffered and inversely proportional to the bandwidth. That is `O(data/bandwidth)`. +* The bandwidth is that available from the host to S3: other work in the same +process, server or network at the time of upload may increase the upload time, +hence the duration of the `close()` call. +* If a process uploading data fails before `OutputStream.close()` is called, +all data is lost. +* The disks hosting temporary directories defined in `fs.s3a.buffer.dir` must +have the capacity to store the entire buffered file. + +Put succinctly: the further the process is from the S3 endpoint, or the smaller +the EC-hosted VM is, the longer it will take work to complete. + +This can create problems in application code: + +* Code often assumes that the `close()` call is fast; + the delays can create bottlenecks in operations. +* Very slow uploads sometimes cause applications to time out. (generally, +threads blocking during the upload stop reporting progress, so trigger timeouts) +* Streaming very large amounts of data may consume all disk space before the upload begins. + + +Work to addess this began in Hadoop 2.7 with the `S3AFastOutputStream` + HADOOP-11183 ( https://issues.apache.org/jira/browse/HADOOP-11183 ), and +has continued with ` S3ABlockOutputStream` + HADOOP-13560 ( https://issues.apache.org/jira/browse/HADOOP-13560 ). + + +This adds an alternative output stream, "S3a Fast Upload" which: + +1. Always uploads large files as blocks with the size set by + `fs.s3a.multipart.size`. That is: the threshold at which multipart uploads + begin and the size of each upload are identical. +1. Buffers blocks to disk (default) or in on-heap or off-heap memory. +1. Uploads blocks in parallel in background threads. +1. Begins uploading blocks as soon as the buffered data exceeds this partition + size. +1. When buffering data to disk, uses the directory/directories listed in + `fs.s3a.buffer.dir`. The size of data which can be buffered is limited + to the available disk space. +1. Generates output statistics as metrics on the filesystem, including + statistics of active and pending block uploads. +1. Has the time to `close()` set by the amount of remaning data to upload, rather + than the total size of the file. + +With incremental writes of blocks, "S3A fast upload" offers an upload +time at least as fast as the "classic" mechanism, with significant benefits +on long-lived output streams, and when very large amounts of data are generated. +The in memory buffering mechanims may also offer speedup when running adjacent to +S3 endpoints, as disks are not used for intermediate data storage. + + +```xml +<property> + <name>fs.s3a.fast.upload</name> + <value>true</value> + <description> + Use the incremental block upload mechanism with + the buffering mechanism set in fs.s3a.fast.upload.buffer. + The number of threads performing uploads in the filesystem is defined + by fs.s3a.threads.max; the queue of waiting uploads limited by + fs.s3a.max.total.tasks. + The size of each buffer is set by fs.s3a.multipart.size. + </description> +</property> + +<property> + <name>fs.s3a.fast.upload.buffer</name> + <value>disk</value> + <description> + The buffering mechanism to use when using S3A fast upload + (fs.s3a.fast.upload=true). Values: disk, array, bytebuffer. + This configuration option has no effect if fs.s3a.fast.upload is false. + + "disk" will use the directories listed in fs.s3a.buffer.dir as + the location(s) to save data prior to being uploaded. + + "array" uses arrays in the JVM heap + + "bytebuffer" uses off-heap memory within the JVM. + + Both "array" and "bytebuffer" will consume memory in a single stream up to the number + of blocks set by: + + fs.s3a.multipart.size * fs.s3a.fast.upload.active.blocks. + + If using either of these mechanisms, keep this value low + + The total number of threads performing work across all threads is set by + fs.s3a.threads.max, with fs.s3a.max.total.tasks values setting the number of queued + work items. + </description> +</property> + +<property> + <name>fs.s3a.multipart.size</name> + <value>104857600</value> + <description> + How big (in bytes) to split upload or copy operations up into. + </description> +</property> + +<property> + <name>fs.s3a.fast.upload.active.blocks</name> + <value>8</value> + <description> + Maximum Number of blocks a single output stream can have + active (uploading, or queued to the central FileSystem + instance's pool of queued operations. + + This stops a single stream overloading the shared thread pool. + </description> +</property> +``` + +* Notes * + +* If the amount of data written to a stream is below that set in `fs.s3a.multipart.size`, +the upload is performed in the `OutputStream.close()` operation —as with +the original output stream. + +* The published Hadoop metrics monitor include live queue length and +upload operation counts, so identifying when there is a backlog of work/ +a mismatch between data generation rates and network bandwidth. Per-stream +statistics can also be logged by calling `toString()` on the current stream. + +* Incremental writes are not visible; the object can only be listed +or read when the multipart operation completes in the `close()` call, which +will block until the upload is completed. + + +#### <a name="s3a_fast_upload_disk"></a>Fast Upload with Disk Buffers `fs.s3a.fast.upload.buffer=disk` + +When `fs.s3a.fast.upload.buffer` is set to `disk`, all data is buffered +to local hard disks prior to upload. This minimizes the amount of memory +consumed, and so eliminates heap size as the limiting factor in queued uploads +—exactly as the original "direct to disk" buffering used when +`fs.s3a.fast.upload=false`. + + +```xml +<property> + <name>fs.s3a.fast.upload</name> + <value>true</value> +</property> + +<property> + <name>fs.s3a.fast.upload.buffer</name> + <value>disk</value> +</property> + +``` + + +#### <a name="s3a_fast_upload_bytebuffer"></a>Fast Upload with ByteBuffers: `fs.s3a.fast.upload.buffer=bytebuffer` + +When `fs.s3a.fast.upload.buffer` is set to `bytebuffer`, all data is buffered +in "Direct" ByteBuffers prior to upload. This may be faster than buffering to disk, +and, if disk space is small (for example, tiny EC2 VMs), there may not +be much disk space to buffer with. + +The ByteBuffers are created in the memory of the JVM, but not in the Java Heap itself. +The amount of data which can be buffered is +limited by the Java runtime, the operating system, and, for YARN applications, +the amount of memory requested for each container. + +The slower the write bandwidth to S3, the greater the risk of running out +of memory. + + +```xml +<property> + <name>fs.s3a.fast.upload</name> + <value>true</value> +</property> + +<property> + <name>fs.s3a.fast.upload.buffer</name> + <value>bytebuffer</value> +</property> +``` + +#### <a name="s3a_fast_upload_array"></a>Fast Upload with Arrays: `fs.s3a.fast.upload.buffer=array` + +When `fs.s3a.fast.upload.buffer` is set to `array`, all data is buffered +in byte arrays in the JVM's heap prior to upload. +This may be faster than buffering to disk. + +This `array` option is similar to the in-memory-only stream offered in +Hadoop 2.7 with `fs.s3a.fast.upload=true` + +The amount of data which can be buffered is limited by the available +size of the JVM heap heap. The slower the write bandwidth to S3, the greater +the risk of heap overflows. + +```xml +<property> + <name>fs.s3a.fast.upload</name> + <value>true</value> +</property> + +<property> + <name>fs.s3a.fast.upload.buffer</name> + <value>array</value> +</property> + +``` +#### <a name="s3a_fast_upload_thread_tuning"></a>S3A Fast Upload Thread Tuning + End diff – As a (probably better) alternative to my other comments, we could explain the bound on the memory consumption here once and link to it.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user thodemoor commented on a diff in the pull request:

          https://github.com/apache/hadoop/pull/130#discussion_r83385370

          — Diff: hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md —
          @@ -881,40 +881,362 @@ Seoul
          If the wrong endpoint is used, the request may fail. This may be reported as a 301/redirect error,
          or as a 400 Bad Request.

                1. S3AFastOutputStream
          • *Warning: NEW in hadoop 2.7. UNSTABLE, EXPERIMENTAL: use at own risk*
          • <property>
          • <name>fs.s3a.fast.upload</name>
          • <value>false</value>
          • <description>Upload directly from memory instead of buffering to
          • disk first. Memory usage and parallelism can be controlled as up to
          • fs.s3a.multipart.size memory is consumed for each (part)upload actively
          • uploading (fs.s3a.threads.max) or queueing (fs.s3a.max.total.tasks)</description>
          • </property>
          • <property>
          • <name>fs.s3a.fast.buffer.size</name>
          • <value>1048576</value>
          • <description>Size (in bytes) of initial memory buffer allocated for an
          • upload. No effect if fs.s3a.fast.upload is false.</description>
          • </property>
            +### <a name="s3a_fast_upload"></a>Stabilizing: S3A Fast Upload
            +
            +
            +*New in Hadoop 2.7; significantly enhanced in Hadoop 2.9*
            +
            +
            +Because of the nature of the S3 object store, data written to an S3A `OutputStream`
            +is not written incrementally —instead, by default, it is buffered to disk
            +until the stream is closed in its `close()` method.
            +
            +This can make output slow:
            +
            +* The execution time for `OutputStream.close()` is proportional to the amount of data
            +buffered and inversely proportional to the bandwidth. That is `O(data/bandwidth)`.
            +* The bandwidth is that available from the host to S3: other work in the same
            +process, server or network at the time of upload may increase the upload time,
            +hence the duration of the `close()` call.
            +* If a process uploading data fails before `OutputStream.close()` is called,
            +all data is lost.
            +* The disks hosting temporary directories defined in `fs.s3a.buffer.dir` must
            +have the capacity to store the entire buffered file.
            +
            +Put succinctly: the further the process is from the S3 endpoint, or the smaller
            +the EC-hosted VM is, the longer it will take work to complete.
            +
            +This can create problems in application code:
            +
            +* Code often assumes that the `close()` call is fast;
            + the delays can create bottlenecks in operations.
            +* Very slow uploads sometimes cause applications to time out. (generally,
            +threads blocking during the upload stop reporting progress, so trigger timeouts)
            +* Streaming very large amounts of data may consume all disk space before the upload begins.
            +
            +
            +Work to addess this began in Hadoop 2.7 with the `S3AFastOutputStream`
            +HADOOP-11183(https://issues.apache.org/jira/browse/HADOOP-11183), and
            +has continued with ` S3ABlockOutputStream`
            +HADOOP-13560(https://issues.apache.org/jira/browse/HADOOP-13560).
            +
            +
            +This adds an alternative output stream, "S3a Fast Upload" which:
            +
            +1. Always uploads large files as blocks with the size set by
            + `fs.s3a.multipart.size`. That is: the threshold at which multipart uploads
            + begin and the size of each upload are identical.
            +1. Buffers blocks to disk (default) or in on-heap or off-heap memory.
            +1. Uploads blocks in parallel in background threads.
            +1. Begins uploading blocks as soon as the buffered data exceeds this partition
            + size.
            +1. When buffering data to disk, uses the directory/directories listed in
            + `fs.s3a.buffer.dir`. The size of data which can be buffered is limited
            + to the available disk space.
            +1. Generates output statistics as metrics on the filesystem, including
            + statistics of active and pending block uploads.
            +1. Has the time to `close()` set by the amount of remaning data to upload, rather
            + than the total size of the file.
            +
            +With incremental writes of blocks, "S3A fast upload" offers an upload
            +time at least as fast as the "classic" mechanism, with significant benefits
            +on long-lived output streams, and when very large amounts of data are generated.
            +The in memory buffering mechanims may also offer speedup when running adjacent to
            +S3 endpoints, as disks are not used for intermediate data storage.
            +
            +
            +```xml
            +<property>
            + <name>fs.s3a.fast.upload</name>
            + <value>true</value>
            + <description>
            + Use the incremental block upload mechanism with
            + the buffering mechanism set in fs.s3a.fast.upload.buffer.
            + The number of threads performing uploads in the filesystem is defined
            + by fs.s3a.threads.max; the queue of waiting uploads limited by
            + fs.s3a.max.total.tasks.
            + The size of each buffer is set by fs.s3a.multipart.size.
            + </description>
            +</property>
            +
            +<property>
            + <name>fs.s3a.fast.upload.buffer</name>
            + <value>disk</value>
            + <description>
            + The buffering mechanism to use when using S3A fast upload
            + (fs.s3a.fast.upload=true). Values: disk, array, bytebuffer.
            + This configuration option has no effect if fs.s3a.fast.upload is false.
            +
            + "disk" will use the directories listed in fs.s3a.buffer.dir as
            + the location(s) to save data prior to being uploaded.
            +
            + "array" uses arrays in the JVM heap
            +
            + "bytebuffer" uses off-heap memory within the JVM.
            +
            + Both "array" and "bytebuffer" will consume memory in a single stream up to the number
            + of blocks set by:
            +
            + fs.s3a.multipart.size * fs.s3a.fast.upload.active.blocks.
            +
            + If using either of these mechanisms, keep this value low
            +
            + The total number of threads performing work across all threads is set by
            + fs.s3a.threads.max, with fs.s3a.max.total.tasks values setting the number of queued
            + work items.
            + </description>
            +</property>
            +
            +<property>
            + <name>fs.s3a.multipart.size</name>
            + <value>104857600</value>
            + <description>
            + How big (in bytes) to split upload or copy operations up into.
            + </description>
            +</property>
            +
            +<property>
            + <name>fs.s3a.fast.upload.active.blocks</name>
            + <value>8</value>
            + <description>
            + Maximum Number of blocks a single output stream can have
            + active (uploading, or queued to the central FileSystem
            + instance's pool of queued operations.
            +
            + This stops a single stream overloading the shared thread pool.
            + </description>
            +</property>
            +```
            +
            +*Notes*
            +
            +* If the amount of data written to a stream is below that set in `fs.s3a.multipart.size`,
            +the upload is performed in the `OutputStream.close()` operation —as with
            +the original output stream.
            +
            +* The published Hadoop metrics monitor include live queue length and
            +upload operation counts, so identifying when there is a backlog of work/
            +a mismatch between data generation rates and network bandwidth. Per-stream
            +statistics can also be logged by calling `toString()` on the current stream.
            +
            +* Incremental writes are not visible; the object can only be listed
            +or read when the multipart operation completes in the `close()` call, which
            +will block until the upload is completed.
            +
            +
            +#### <a name="s3a_fast_upload_disk"></a>Fast Upload with Disk Buffers `fs.s3a.fast.upload.buffer=disk`
            +
            +When `fs.s3a.fast.upload.buffer` is set to `disk`, all data is buffered
            +to local hard disks prior to upload. This minimizes the amount of memory
            +consumed, and so eliminates heap size as the limiting factor in queued uploads
            +—exactly as the original "direct to disk" buffering used when
            +`fs.s3a.fast.upload=false`.
            +
            +
            +```xml
            +<property>
            + <name>fs.s3a.fast.upload</name>
            + <value>true</value>
            +</property>
            +
            +<property>
            + <name>fs.s3a.fast.upload.buffer</name>
            + <value>disk</value>
            +</property>
            +
            +```
            +
            +
            +#### <a name="s3a_fast_upload_bytebuffer"></a>Fast Upload with ByteBuffers: `fs.s3a.fast.upload.buffer=bytebuffer`
            +
            +When `fs.s3a.fast.upload.buffer` is set to `bytebuffer`, all data is buffered
            +in "Direct" ByteBuffers prior to upload. This may be faster than buffering to disk,
            +and, if disk space is small (for example, tiny EC2 VMs), there may not
            +be much disk space to buffer with.
            +
            +The ByteBuffers are created in the memory of the JVM, but not in the Java Heap itself.
            +The amount of data which can be buffered is
            +limited by the Java runtime, the operating system, and, for YARN applications,
            +the amount of memory requested for each container.
            +
            +The slower the write bandwidth to S3, the greater the risk of running out
            +of memory.
            +
            +
            +```xml
            +<property>
            + <name>fs.s3a.fast.upload</name>
            + <value>true</value>
            +</property>
            +
            +<property>
            + <name>fs.s3a.fast.upload.buffer</name>
            + <value>bytebuffer</value>
            +</property>
            +```
            +
            +#### <a name="s3a_fast_upload_array"></a>Fast Upload with Arrays: `fs.s3a.fast.upload.buffer=array`
            +
            +When `fs.s3a.fast.upload.buffer` is set to `array`, all data is buffered
            +in byte arrays in the JVM's heap prior to upload.
            +This may be faster than buffering to disk.
            +
            +This `array` option is similar to the in-memory-only stream offered in
            +Hadoop 2.7 with `fs.s3a.fast.upload=true`
            +
            +The amount of data which can be buffered is limited by the available
            +size of the JVM heap heap. The slower the write bandwidth to S3, the greater
            +the risk of heap overflows.
            +
            +```xml
            +<property>
            + <name>fs.s3a.fast.upload</name>
            + <value>true</value>
            +</property>
            +
            +<property>
            + <name>fs.s3a.fast.upload.buffer</name>
            + <value>array</value>
            +</property>
            +
            +```
            +#### <a name="s3a_fast_upload_thread_tuning"></a>S3A Fast Upload Thread Tuning
            +
            +Both the [Array](#s3a_fast_upload_array) and [Byte buffer](#s3a_fast_upload_bytebuffer)
            +buffer mechanisms can consume very large amounts of memory, on-heap or
            +off-heap respectively. The [disk buffer](#s3a_fast_upload_disk) mechanism
            +does not use much memory up, but will consume hard disk capacity.
            +
            +If there are many output streams being written to in a single process, the
            +amount of memory or disk used is the multiple of all stream's active memory/disk use.
            +
            +Careful tuning may be needed to reduce the risk of running out memory, especially
            +if the data is buffered in memory.
            +
            +There are a number parameters which can be tuned:
            +
            +1. The total number of threads available in the filesystem for data
            +uploads or any other queued filesystem operation. This is set in
            +`fs.s3a.threads.max`
            +
            +1. The number of operations which can be queued for execution:, *awaiting
            +a thread*: `fs.s3a.max.total.tasks`
            +
            +1. The number of blocks which a single output stream can have active,
            +that is: being uploaded by a thread, or queued in the filesystem thread queue:
            +`fs.s3a.fast.upload.active.blocks`
            +
            +1. How long an idle thread can stay in the thread pool before it is retired: `fs.s3a.threads.keepalivetime`
            +
            +
            +When the maximum allowed number of active blocks of a single stream is reached,
            +no more blocks can be uploaded from that stream until one or more of those active
            +blocks' uploads completes. That is: a `write()` call which would trigger an upload
            +of a now full datablock, will instead block until there is capacity in the queue.
            +
            +How does that come together?
            +
            +* As the pool of threads set in `fs.s3a.threads.max` is shared (and intended
            +to be used across all threads), a larger number here can allow for more
            +parallel operations. However, as uploads require network bandwidth, adding more
            +threads does not guarantee speedup.
            +
            +* The extra queue of tasks for the thread pool (`fs.s3a.max.total.tasks`)
            +covers all ongoing background S3A operations (future plans include: parallelized
            +rename operations, asynchronous directory operations).
            +
            +* When using memory buffering, a small value of `fs.s3a.fast.upload.active.blocks`
            +limits the amount of memory which can be consumed per stream.
            +
            +* When using disk buffering a larger value of `fs.s3a.fast.upload.active.blocks`
            +does not consume much memory. But it may result in a large number of blocks to
            +compete with other filesystem operations.
            +
            +
            +We recommend a low value of `fs.s3a.fast.upload.active.blocks`; enough
            +to start background upload without overloading other parts of the system,
            +then experiment to see if higher values deliver more throughtput —especially
            +from VMs running on EC2.
            +
            +```xml
            +
            +<property>
            + <name>fs.s3a.fast.upload.active.blocks</name>
            + <value>4</value>
            + <description>
            + Maximum Number of blocks a single output stream can have
            + active (uploading, or queued to the central FileSystem
            + instance's pool of queued operations.
            +
            + This stops a single stream overloading the shared thread pool.
            + </description>
            +</property>
            +
            +<property>
            + <name>fs.s3a.threads.max</name>
            + <value>10</value>
            + <description>The total number of threads available in the filesystem for data
            + uploads or any other queued filesystem operation.</description>
            +</property>
            +
            +<property>
            + <name>fs.s3a.max.total.tasks</name>
            + <value>5</value>
            + <description>The number of operations which can be queued for execution</description>
            +</property>
            +
            +<property>
            + <name>fs.s3a.threads.keepalivetime</name>
            + <value>60</value>
            + <description>Number of seconds a thread can be idle before being
            + terminated.</description>
            +</property>
            +
            +```
            +
            +
            +#### <a name="s3a_multipart_purge"></a>Cleaning up After Incremental Upload Failures: `fs.s3a.multipart.purge`
            +
            +
            +If an incremental streaming operation is interrupted, there may be
            +intermediate partitions uploaded to S3 —data which will be billed for.
            +
            +These charges can be reduced by enabling `fs.s3a.multipart.purge`,
            +and setting a purge time in seconds, such as 86400 seconds —24 hours, after
            +which the S3 service automatically deletes outstanding multipart
              • End diff –

          To me, the wording here gives the impression this is a server-side operation but the purging happens on the client by listing all uploads and then sending a delete call with the ones to be purged. Consequently, this can cause a (slight) delay when instantiating an s3a FS instance and there are lots of active uploads (to purge).

          Show
          githubbot ASF GitHub Bot added a comment - Github user thodemoor commented on a diff in the pull request: https://github.com/apache/hadoop/pull/130#discussion_r83385370 — Diff: hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md — @@ -881,40 +881,362 @@ Seoul If the wrong endpoint is used, the request may fail. This may be reported as a 301/redirect error, or as a 400 Bad Request. S3AFastOutputStream * Warning: NEW in hadoop 2.7. UNSTABLE, EXPERIMENTAL: use at own risk * <property> <name>fs.s3a.fast.upload</name> <value>false</value> <description>Upload directly from memory instead of buffering to disk first. Memory usage and parallelism can be controlled as up to fs.s3a.multipart.size memory is consumed for each (part)upload actively uploading (fs.s3a.threads.max) or queueing (fs.s3a.max.total.tasks)</description> </property> <property> <name>fs.s3a.fast.buffer.size</name> <value>1048576</value> <description>Size (in bytes) of initial memory buffer allocated for an upload. No effect if fs.s3a.fast.upload is false.</description> </property> +### <a name="s3a_fast_upload"></a>Stabilizing: S3A Fast Upload + + +* New in Hadoop 2.7; significantly enhanced in Hadoop 2.9 * + + +Because of the nature of the S3 object store, data written to an S3A `OutputStream` +is not written incrementally —instead, by default, it is buffered to disk +until the stream is closed in its `close()` method. + +This can make output slow: + +* The execution time for `OutputStream.close()` is proportional to the amount of data +buffered and inversely proportional to the bandwidth. That is `O(data/bandwidth)`. +* The bandwidth is that available from the host to S3: other work in the same +process, server or network at the time of upload may increase the upload time, +hence the duration of the `close()` call. +* If a process uploading data fails before `OutputStream.close()` is called, +all data is lost. +* The disks hosting temporary directories defined in `fs.s3a.buffer.dir` must +have the capacity to store the entire buffered file. + +Put succinctly: the further the process is from the S3 endpoint, or the smaller +the EC-hosted VM is, the longer it will take work to complete. + +This can create problems in application code: + +* Code often assumes that the `close()` call is fast; + the delays can create bottlenecks in operations. +* Very slow uploads sometimes cause applications to time out. (generally, +threads blocking during the upload stop reporting progress, so trigger timeouts) +* Streaming very large amounts of data may consume all disk space before the upload begins. + + +Work to addess this began in Hadoop 2.7 with the `S3AFastOutputStream` + HADOOP-11183 ( https://issues.apache.org/jira/browse/HADOOP-11183 ), and +has continued with ` S3ABlockOutputStream` + HADOOP-13560 ( https://issues.apache.org/jira/browse/HADOOP-13560 ). + + +This adds an alternative output stream, "S3a Fast Upload" which: + +1. Always uploads large files as blocks with the size set by + `fs.s3a.multipart.size`. That is: the threshold at which multipart uploads + begin and the size of each upload are identical. +1. Buffers blocks to disk (default) or in on-heap or off-heap memory. +1. Uploads blocks in parallel in background threads. +1. Begins uploading blocks as soon as the buffered data exceeds this partition + size. +1. When buffering data to disk, uses the directory/directories listed in + `fs.s3a.buffer.dir`. The size of data which can be buffered is limited + to the available disk space. +1. Generates output statistics as metrics on the filesystem, including + statistics of active and pending block uploads. +1. Has the time to `close()` set by the amount of remaning data to upload, rather + than the total size of the file. + +With incremental writes of blocks, "S3A fast upload" offers an upload +time at least as fast as the "classic" mechanism, with significant benefits +on long-lived output streams, and when very large amounts of data are generated. +The in memory buffering mechanims may also offer speedup when running adjacent to +S3 endpoints, as disks are not used for intermediate data storage. + + +```xml +<property> + <name>fs.s3a.fast.upload</name> + <value>true</value> + <description> + Use the incremental block upload mechanism with + the buffering mechanism set in fs.s3a.fast.upload.buffer. + The number of threads performing uploads in the filesystem is defined + by fs.s3a.threads.max; the queue of waiting uploads limited by + fs.s3a.max.total.tasks. + The size of each buffer is set by fs.s3a.multipart.size. + </description> +</property> + +<property> + <name>fs.s3a.fast.upload.buffer</name> + <value>disk</value> + <description> + The buffering mechanism to use when using S3A fast upload + (fs.s3a.fast.upload=true). Values: disk, array, bytebuffer. + This configuration option has no effect if fs.s3a.fast.upload is false. + + "disk" will use the directories listed in fs.s3a.buffer.dir as + the location(s) to save data prior to being uploaded. + + "array" uses arrays in the JVM heap + + "bytebuffer" uses off-heap memory within the JVM. + + Both "array" and "bytebuffer" will consume memory in a single stream up to the number + of blocks set by: + + fs.s3a.multipart.size * fs.s3a.fast.upload.active.blocks. + + If using either of these mechanisms, keep this value low + + The total number of threads performing work across all threads is set by + fs.s3a.threads.max, with fs.s3a.max.total.tasks values setting the number of queued + work items. + </description> +</property> + +<property> + <name>fs.s3a.multipart.size</name> + <value>104857600</value> + <description> + How big (in bytes) to split upload or copy operations up into. + </description> +</property> + +<property> + <name>fs.s3a.fast.upload.active.blocks</name> + <value>8</value> + <description> + Maximum Number of blocks a single output stream can have + active (uploading, or queued to the central FileSystem + instance's pool of queued operations. + + This stops a single stream overloading the shared thread pool. + </description> +</property> +``` + +* Notes * + +* If the amount of data written to a stream is below that set in `fs.s3a.multipart.size`, +the upload is performed in the `OutputStream.close()` operation —as with +the original output stream. + +* The published Hadoop metrics monitor include live queue length and +upload operation counts, so identifying when there is a backlog of work/ +a mismatch between data generation rates and network bandwidth. Per-stream +statistics can also be logged by calling `toString()` on the current stream. + +* Incremental writes are not visible; the object can only be listed +or read when the multipart operation completes in the `close()` call, which +will block until the upload is completed. + + +#### <a name="s3a_fast_upload_disk"></a>Fast Upload with Disk Buffers `fs.s3a.fast.upload.buffer=disk` + +When `fs.s3a.fast.upload.buffer` is set to `disk`, all data is buffered +to local hard disks prior to upload. This minimizes the amount of memory +consumed, and so eliminates heap size as the limiting factor in queued uploads +—exactly as the original "direct to disk" buffering used when +`fs.s3a.fast.upload=false`. + + +```xml +<property> + <name>fs.s3a.fast.upload</name> + <value>true</value> +</property> + +<property> + <name>fs.s3a.fast.upload.buffer</name> + <value>disk</value> +</property> + +``` + + +#### <a name="s3a_fast_upload_bytebuffer"></a>Fast Upload with ByteBuffers: `fs.s3a.fast.upload.buffer=bytebuffer` + +When `fs.s3a.fast.upload.buffer` is set to `bytebuffer`, all data is buffered +in "Direct" ByteBuffers prior to upload. This may be faster than buffering to disk, +and, if disk space is small (for example, tiny EC2 VMs), there may not +be much disk space to buffer with. + +The ByteBuffers are created in the memory of the JVM, but not in the Java Heap itself. +The amount of data which can be buffered is +limited by the Java runtime, the operating system, and, for YARN applications, +the amount of memory requested for each container. + +The slower the write bandwidth to S3, the greater the risk of running out +of memory. + + +```xml +<property> + <name>fs.s3a.fast.upload</name> + <value>true</value> +</property> + +<property> + <name>fs.s3a.fast.upload.buffer</name> + <value>bytebuffer</value> +</property> +``` + +#### <a name="s3a_fast_upload_array"></a>Fast Upload with Arrays: `fs.s3a.fast.upload.buffer=array` + +When `fs.s3a.fast.upload.buffer` is set to `array`, all data is buffered +in byte arrays in the JVM's heap prior to upload. +This may be faster than buffering to disk. + +This `array` option is similar to the in-memory-only stream offered in +Hadoop 2.7 with `fs.s3a.fast.upload=true` + +The amount of data which can be buffered is limited by the available +size of the JVM heap heap. The slower the write bandwidth to S3, the greater +the risk of heap overflows. + +```xml +<property> + <name>fs.s3a.fast.upload</name> + <value>true</value> +</property> + +<property> + <name>fs.s3a.fast.upload.buffer</name> + <value>array</value> +</property> + +``` +#### <a name="s3a_fast_upload_thread_tuning"></a>S3A Fast Upload Thread Tuning + +Both the [Array] (#s3a_fast_upload_array) and [Byte buffer] (#s3a_fast_upload_bytebuffer) +buffer mechanisms can consume very large amounts of memory, on-heap or +off-heap respectively. The [disk buffer] (#s3a_fast_upload_disk) mechanism +does not use much memory up, but will consume hard disk capacity. + +If there are many output streams being written to in a single process, the +amount of memory or disk used is the multiple of all stream's active memory/disk use. + +Careful tuning may be needed to reduce the risk of running out memory, especially +if the data is buffered in memory. + +There are a number parameters which can be tuned: + +1. The total number of threads available in the filesystem for data +uploads or any other queued filesystem operation . This is set in +`fs.s3a.threads.max` + +1. The number of operations which can be queued for execution:, *awaiting +a thread*: `fs.s3a.max.total.tasks` + +1. The number of blocks which a single output stream can have active, +that is: being uploaded by a thread, or queued in the filesystem thread queue: +`fs.s3a.fast.upload.active.blocks` + +1. How long an idle thread can stay in the thread pool before it is retired: `fs.s3a.threads.keepalivetime` + + +When the maximum allowed number of active blocks of a single stream is reached, +no more blocks can be uploaded from that stream until one or more of those active +blocks' uploads completes. That is: a `write()` call which would trigger an upload +of a now full datablock, will instead block until there is capacity in the queue. + +How does that come together? + +* As the pool of threads set in `fs.s3a.threads.max` is shared (and intended +to be used across all threads), a larger number here can allow for more +parallel operations. However, as uploads require network bandwidth, adding more +threads does not guarantee speedup. + +* The extra queue of tasks for the thread pool (`fs.s3a.max.total.tasks`) +covers all ongoing background S3A operations (future plans include: parallelized +rename operations, asynchronous directory operations). + +* When using memory buffering, a small value of `fs.s3a.fast.upload.active.blocks` +limits the amount of memory which can be consumed per stream. + +* When using disk buffering a larger value of `fs.s3a.fast.upload.active.blocks` +does not consume much memory. But it may result in a large number of blocks to +compete with other filesystem operations. + + +We recommend a low value of `fs.s3a.fast.upload.active.blocks`; enough +to start background upload without overloading other parts of the system, +then experiment to see if higher values deliver more throughtput —especially +from VMs running on EC2. + +```xml + +<property> + <name>fs.s3a.fast.upload.active.blocks</name> + <value>4</value> + <description> + Maximum Number of blocks a single output stream can have + active (uploading, or queued to the central FileSystem + instance's pool of queued operations. + + This stops a single stream overloading the shared thread pool. + </description> +</property> + +<property> + <name>fs.s3a.threads.max</name> + <value>10</value> + <description>The total number of threads available in the filesystem for data + uploads or any other queued filesystem operation .</description> +</property> + +<property> + <name>fs.s3a.max.total.tasks</name> + <value>5</value> + <description>The number of operations which can be queued for execution</description> +</property> + +<property> + <name>fs.s3a.threads.keepalivetime</name> + <value>60</value> + <description>Number of seconds a thread can be idle before being + terminated.</description> +</property> + +``` + + +#### <a name="s3a_multipart_purge"></a>Cleaning up After Incremental Upload Failures: `fs.s3a.multipart.purge` + + +If an incremental streaming operation is interrupted, there may be +intermediate partitions uploaded to S3 —data which will be billed for. + +These charges can be reduced by enabling `fs.s3a.multipart.purge`, +and setting a purge time in seconds, such as 86400 seconds —24 hours, after +which the S3 service automatically deletes outstanding multipart End diff – To me, the wording here gives the impression this is a server-side operation but the purging happens on the client by listing all uploads and then sending a delete call with the ones to be purged. Consequently, this can cause a (slight) delay when instantiating an s3a FS instance and there are lots of active uploads (to purge).
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user steveloughran commented on a diff in the pull request:

          https://github.com/apache/hadoop/pull/130#discussion_r83414208

          — Diff: hadoop-common-project/hadoop-common/src/main/resources/core-default.xml —
          @@ -1095,10 +1102,50 @@
          <property>
          <name>fs.s3a.fast.upload</name>
          <value>false</value>

          • <description>Upload directly from memory instead of buffering to
          • disk first. Memory usage and parallelism can be controlled as up to
          • fs.s3a.multipart.size memory is consumed for each (part)upload actively
          • uploading (fs.s3a.threads.max) or queueing (fs.s3a.max.total.tasks)</description>
            + <description>
            + Use the incremental block-based fast upload mechanism with
            + the buffering mechanism set in fs.s3a.fast.upload.buffer.
            + </description>
            +</property>
            +
            +<property>
            + <name>fs.s3a.fast.upload.buffer</name>
            + <value>disk</value>
            + <description>
            + The buffering mechanism to use when using S3A fast upload
            + (fs.s3a.fast.upload=true). Values: disk, array, bytebuffer.
            + This configuration option has no effect if fs.s3a.fast.upload is false.
            +
            + "disk" will use the directories listed in fs.s3a.buffer.dir as
            + the location(s) to save data prior to being uploaded.
            +
            + "array" uses arrays in the JVM heap
            +
            + "bytebuffer" uses off-heap memory within the JVM.
            +
            + Both "array" and "bytebuffer" will consume memory in a single stream up to the number
            + of blocks set by:
            +
            + fs.s3a.multipart.size * fs.s3a.fast.upload.active.blocks.
            +
            + If using either of these mechanisms, keep this value low
            +
            + The total number of threads performing work across all threads is set by
            + fs.s3a.threads.max, with fs.s3a.max.total.tasks values setting the number of queued
            + work items.
              • End diff –

          you know, now that you can have a queue per stream, it could be set to something
          bigger. This is something we could look at in the docs, leaving out of the XML so as
          to have a single topic. This phrase here describes the number of active threads, which
          is different —and will be more so once there's other work (COPY, DELETE) going on there.

          So: wont change here

          Show
          githubbot ASF GitHub Bot added a comment - Github user steveloughran commented on a diff in the pull request: https://github.com/apache/hadoop/pull/130#discussion_r83414208 — Diff: hadoop-common-project/hadoop-common/src/main/resources/core-default.xml — @@ -1095,10 +1102,50 @@ <property> <name>fs.s3a.fast.upload</name> <value>false</value> <description>Upload directly from memory instead of buffering to disk first. Memory usage and parallelism can be controlled as up to fs.s3a.multipart.size memory is consumed for each (part)upload actively uploading (fs.s3a.threads.max) or queueing (fs.s3a.max.total.tasks)</description> + <description> + Use the incremental block-based fast upload mechanism with + the buffering mechanism set in fs.s3a.fast.upload.buffer. + </description> +</property> + +<property> + <name>fs.s3a.fast.upload.buffer</name> + <value>disk</value> + <description> + The buffering mechanism to use when using S3A fast upload + (fs.s3a.fast.upload=true). Values: disk, array, bytebuffer. + This configuration option has no effect if fs.s3a.fast.upload is false. + + "disk" will use the directories listed in fs.s3a.buffer.dir as + the location(s) to save data prior to being uploaded. + + "array" uses arrays in the JVM heap + + "bytebuffer" uses off-heap memory within the JVM. + + Both "array" and "bytebuffer" will consume memory in a single stream up to the number + of blocks set by: + + fs.s3a.multipart.size * fs.s3a.fast.upload.active.blocks. + + If using either of these mechanisms, keep this value low + + The total number of threads performing work across all threads is set by + fs.s3a.threads.max, with fs.s3a.max.total.tasks values setting the number of queued + work items. End diff – you know, now that you can have a queue per stream, it could be set to something bigger. This is something we could look at in the docs, leaving out of the XML so as to have a single topic. This phrase here describes the number of active threads, which is different —and will be more so once there's other work (COPY, DELETE) going on there. So: wont change here
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user thodemoor commented on a diff in the pull request:

          https://github.com/apache/hadoop/pull/130#discussion_r83415072

          — Diff: hadoop-common-project/hadoop-common/src/main/resources/core-default.xml —
          @@ -1095,10 +1102,50 @@
          <property>
          <name>fs.s3a.fast.upload</name>
          <value>false</value>

          • <description>Upload directly from memory instead of buffering to
          • disk first. Memory usage and parallelism can be controlled as up to
          • fs.s3a.multipart.size memory is consumed for each (part)upload actively
          • uploading (fs.s3a.threads.max) or queueing (fs.s3a.max.total.tasks)</description>
            + <description>
            + Use the incremental block-based fast upload mechanism with
            + the buffering mechanism set in fs.s3a.fast.upload.buffer.
            + </description>
            +</property>
            +
            +<property>
            + <name>fs.s3a.fast.upload.buffer</name>
            + <value>disk</value>
            + <description>
            + The buffering mechanism to use when using S3A fast upload
            + (fs.s3a.fast.upload=true). Values: disk, array, bytebuffer.
            + This configuration option has no effect if fs.s3a.fast.upload is false.
            +
            + "disk" will use the directories listed in fs.s3a.buffer.dir as
            + the location(s) to save data prior to being uploaded.
            +
            + "array" uses arrays in the JVM heap
            +
            + "bytebuffer" uses off-heap memory within the JVM.
            +
            + Both "array" and "bytebuffer" will consume memory in a single stream up to the number
            + of blocks set by:
            +
            + fs.s3a.multipart.size * fs.s3a.fast.upload.active.blocks.
            +
            + If using either of these mechanisms, keep this value low
            +
            + The total number of threads performing work across all threads is set by
            + fs.s3a.threads.max, with fs.s3a.max.total.tasks values setting the number of queued
            + work items.
              • End diff –

          Completely agree. A bit further down I propose to add a single explanation in the javadoc and link to there in the various other locations

          Show
          githubbot ASF GitHub Bot added a comment - Github user thodemoor commented on a diff in the pull request: https://github.com/apache/hadoop/pull/130#discussion_r83415072 — Diff: hadoop-common-project/hadoop-common/src/main/resources/core-default.xml — @@ -1095,10 +1102,50 @@ <property> <name>fs.s3a.fast.upload</name> <value>false</value> <description>Upload directly from memory instead of buffering to disk first. Memory usage and parallelism can be controlled as up to fs.s3a.multipart.size memory is consumed for each (part)upload actively uploading (fs.s3a.threads.max) or queueing (fs.s3a.max.total.tasks)</description> + <description> + Use the incremental block-based fast upload mechanism with + the buffering mechanism set in fs.s3a.fast.upload.buffer. + </description> +</property> + +<property> + <name>fs.s3a.fast.upload.buffer</name> + <value>disk</value> + <description> + The buffering mechanism to use when using S3A fast upload + (fs.s3a.fast.upload=true). Values: disk, array, bytebuffer. + This configuration option has no effect if fs.s3a.fast.upload is false. + + "disk" will use the directories listed in fs.s3a.buffer.dir as + the location(s) to save data prior to being uploaded. + + "array" uses arrays in the JVM heap + + "bytebuffer" uses off-heap memory within the JVM. + + Both "array" and "bytebuffer" will consume memory in a single stream up to the number + of blocks set by: + + fs.s3a.multipart.size * fs.s3a.fast.upload.active.blocks. + + If using either of these mechanisms, keep this value low + + The total number of threads performing work across all threads is set by + fs.s3a.threads.max, with fs.s3a.max.total.tasks values setting the number of queued + work items. End diff – Completely agree. A bit further down I propose to add a single explanation in the javadoc and link to there in the various other locations
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user pieterreuse commented on a diff in the pull request:

          https://github.com/apache/hadoop/pull/130#discussion_r83416183

          — Diff: hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java —
          @@ -118,21 +126,37 @@
          private long partSize;
          private boolean enableMultiObjectsDelete;
          private TransferManager transfers;

          • private ExecutorService threadPoolExecutor;
            + private ListeningExecutorService threadPoolExecutor;
            private long multiPartThreshold;
            public static final Logger LOG = LoggerFactory.getLogger(S3AFileSystem.class);
            + private static final Logger PROGRESS =
            + LoggerFactory.getLogger("org.apache.hadoop.fs.s3a.S3AFileSystem.Progress");
            + private LocalDirAllocator directoryAllocator;
            private CannedAccessControlList cannedACL;
            private String serverSideEncryptionAlgorithm;
            private S3AInstrumentation instrumentation;
            private S3AStorageStatistics storageStatistics;
            private long readAhead;
            private S3AInputPolicy inputPolicy;
          • private static final AtomicBoolean warnedOfCoreThreadDeprecation =
          • new AtomicBoolean(false);
            private final AtomicBoolean closed = new AtomicBoolean(false);

          // The maximum number of entries that can be deleted in any call to s3
          private static final int MAX_ENTRIES_TO_DELETE = 1000;
          + private boolean blockUploadEnabled;
          + private String blockOutputBuffer;
          + private S3ADataBlocks.BlockFactory blockFactory;
          + private int blockOutputActiveBlocks;
          +
          + /*
          + * Register Deprecated options.
          + */
          + static {
          + Configuration.addDeprecations(new Configuration.DeprecationDelta[]{
          + new Configuration.DeprecationDelta("fs.s3a.threads.core",
          + null,
          — End diff –

          I'm not familiar with DeprecationDelta's, but this null value gave rise to a nullpointerexception on *all* unit tests when fs.s3a.threads.core was in my config. Replacing this null with "" (empty string) resolved my issue, but I'm not 100% sure that is the right thing to do here.

          Show
          githubbot ASF GitHub Bot added a comment - Github user pieterreuse commented on a diff in the pull request: https://github.com/apache/hadoop/pull/130#discussion_r83416183 — Diff: hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java — @@ -118,21 +126,37 @@ private long partSize; private boolean enableMultiObjectsDelete; private TransferManager transfers; private ExecutorService threadPoolExecutor; + private ListeningExecutorService threadPoolExecutor; private long multiPartThreshold; public static final Logger LOG = LoggerFactory.getLogger(S3AFileSystem.class); + private static final Logger PROGRESS = + LoggerFactory.getLogger("org.apache.hadoop.fs.s3a.S3AFileSystem.Progress"); + private LocalDirAllocator directoryAllocator; private CannedAccessControlList cannedACL; private String serverSideEncryptionAlgorithm; private S3AInstrumentation instrumentation; private S3AStorageStatistics storageStatistics; private long readAhead; private S3AInputPolicy inputPolicy; private static final AtomicBoolean warnedOfCoreThreadDeprecation = new AtomicBoolean(false); private final AtomicBoolean closed = new AtomicBoolean(false); // The maximum number of entries that can be deleted in any call to s3 private static final int MAX_ENTRIES_TO_DELETE = 1000; + private boolean blockUploadEnabled; + private String blockOutputBuffer; + private S3ADataBlocks.BlockFactory blockFactory; + private int blockOutputActiveBlocks; + + /* + * Register Deprecated options. + */ + static { + Configuration.addDeprecations(new Configuration.DeprecationDelta[]{ + new Configuration.DeprecationDelta("fs.s3a.threads.core", + null, — End diff – I'm not familiar with DeprecationDelta's, but this null value gave rise to a nullpointerexception on * all * unit tests when fs.s3a.threads.core was in my config. Replacing this null with "" (empty string) resolved my issue, but I'm not 100% sure that is the right thing to do here.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user steveloughran commented on a diff in the pull request:

          https://github.com/apache/hadoop/pull/130#discussion_r83419491

          — Diff: hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md —
          @@ -881,40 +881,362 @@ Seoul
          If the wrong endpoint is used, the request may fail. This may be reported as a 301/redirect error,
          or as a 400 Bad Request.

                1. S3AFastOutputStream
          • *Warning: NEW in hadoop 2.7. UNSTABLE, EXPERIMENTAL: use at own risk*
          • <property>
          • <name>fs.s3a.fast.upload</name>
          • <value>false</value>
          • <description>Upload directly from memory instead of buffering to
          • disk first. Memory usage and parallelism can be controlled as up to
          • fs.s3a.multipart.size memory is consumed for each (part)upload actively
          • uploading (fs.s3a.threads.max) or queueing (fs.s3a.max.total.tasks)</description>
          • </property>
          • <property>
          • <name>fs.s3a.fast.buffer.size</name>
          • <value>1048576</value>
          • <description>Size (in bytes) of initial memory buffer allocated for an
          • upload. No effect if fs.s3a.fast.upload is false.</description>
          • </property>
            +### <a name="s3a_fast_upload"></a>Stabilizing: S3A Fast Upload
            +
            +
            +*New in Hadoop 2.7; significantly enhanced in Hadoop 2.9*
            +
            +
            +Because of the nature of the S3 object store, data written to an S3A `OutputStream`
            +is not written incrementally —instead, by default, it is buffered to disk
            +until the stream is closed in its `close()` method.
            +
            +This can make output slow:
            +
            +* The execution time for `OutputStream.close()` is proportional to the amount of data
            +buffered and inversely proportional to the bandwidth. That is `O(data/bandwidth)`.
            +* The bandwidth is that available from the host to S3: other work in the same
            +process, server or network at the time of upload may increase the upload time,
            +hence the duration of the `close()` call.
            +* If a process uploading data fails before `OutputStream.close()` is called,
            +all data is lost.
            +* The disks hosting temporary directories defined in `fs.s3a.buffer.dir` must
            +have the capacity to store the entire buffered file.
            +
            +Put succinctly: the further the process is from the S3 endpoint, or the smaller
            +the EC-hosted VM is, the longer it will take work to complete.
            +
            +This can create problems in application code:
            +
            +* Code often assumes that the `close()` call is fast;
            + the delays can create bottlenecks in operations.
            +* Very slow uploads sometimes cause applications to time out. (generally,
            +threads blocking during the upload stop reporting progress, so trigger timeouts)
            +* Streaming very large amounts of data may consume all disk space before the upload begins.
            +
            +
            +Work to addess this began in Hadoop 2.7 with the `S3AFastOutputStream`
            +HADOOP-11183(https://issues.apache.org/jira/browse/HADOOP-11183), and
            +has continued with ` S3ABlockOutputStream`
            +HADOOP-13560(https://issues.apache.org/jira/browse/HADOOP-13560).
            +
            +
            +This adds an alternative output stream, "S3a Fast Upload" which:
            +
            +1. Always uploads large files as blocks with the size set by
            + `fs.s3a.multipart.size`. That is: the threshold at which multipart uploads
            + begin and the size of each upload are identical.
            +1. Buffers blocks to disk (default) or in on-heap or off-heap memory.
            +1. Uploads blocks in parallel in background threads.
            +1. Begins uploading blocks as soon as the buffered data exceeds this partition
            + size.
            +1. When buffering data to disk, uses the directory/directories listed in
            + `fs.s3a.buffer.dir`. The size of data which can be buffered is limited
            + to the available disk space.
            +1. Generates output statistics as metrics on the filesystem, including
            + statistics of active and pending block uploads.
            +1. Has the time to `close()` set by the amount of remaning data to upload, rather
            + than the total size of the file.
            +
            +With incremental writes of blocks, "S3A fast upload" offers an upload
            +time at least as fast as the "classic" mechanism, with significant benefits
            +on long-lived output streams, and when very large amounts of data are generated.
            +The in memory buffering mechanims may also offer speedup when running adjacent to
            +S3 endpoints, as disks are not used for intermediate data storage.
            +
            +
            +```xml
            +<property>
            + <name>fs.s3a.fast.upload</name>
            + <value>true</value>
            + <description>
            + Use the incremental block upload mechanism with
            + the buffering mechanism set in fs.s3a.fast.upload.buffer.
            + The number of threads performing uploads in the filesystem is defined
            + by fs.s3a.threads.max; the queue of waiting uploads limited by
            + fs.s3a.max.total.tasks.
            + The size of each buffer is set by fs.s3a.multipart.size.
            + </description>
            +</property>
            +
            +<property>
            + <name>fs.s3a.fast.upload.buffer</name>
            + <value>disk</value>
            + <description>
            + The buffering mechanism to use when using S3A fast upload
            + (fs.s3a.fast.upload=true). Values: disk, array, bytebuffer.
            + This configuration option has no effect if fs.s3a.fast.upload is false.
            +
            + "disk" will use the directories listed in fs.s3a.buffer.dir as
            + the location(s) to save data prior to being uploaded.
            +
            + "array" uses arrays in the JVM heap
            +
            + "bytebuffer" uses off-heap memory within the JVM.
            +
            + Both "array" and "bytebuffer" will consume memory in a single stream up to the number
            + of blocks set by:
            +
            + fs.s3a.multipart.size * fs.s3a.fast.upload.active.blocks.
            +
            + If using either of these mechanisms, keep this value low
            +
            + The total number of threads performing work across all threads is set by
            + fs.s3a.threads.max, with fs.s3a.max.total.tasks values setting the number of queued
            + work items.
              • End diff –

          ..again, not changing it in either place, as once renames() parallelize, life gets more complex

          Show
          githubbot ASF GitHub Bot added a comment - Github user steveloughran commented on a diff in the pull request: https://github.com/apache/hadoop/pull/130#discussion_r83419491 — Diff: hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md — @@ -881,40 +881,362 @@ Seoul If the wrong endpoint is used, the request may fail. This may be reported as a 301/redirect error, or as a 400 Bad Request. S3AFastOutputStream * Warning: NEW in hadoop 2.7. UNSTABLE, EXPERIMENTAL: use at own risk * <property> <name>fs.s3a.fast.upload</name> <value>false</value> <description>Upload directly from memory instead of buffering to disk first. Memory usage and parallelism can be controlled as up to fs.s3a.multipart.size memory is consumed for each (part)upload actively uploading (fs.s3a.threads.max) or queueing (fs.s3a.max.total.tasks)</description> </property> <property> <name>fs.s3a.fast.buffer.size</name> <value>1048576</value> <description>Size (in bytes) of initial memory buffer allocated for an upload. No effect if fs.s3a.fast.upload is false.</description> </property> +### <a name="s3a_fast_upload"></a>Stabilizing: S3A Fast Upload + + +* New in Hadoop 2.7; significantly enhanced in Hadoop 2.9 * + + +Because of the nature of the S3 object store, data written to an S3A `OutputStream` +is not written incrementally —instead, by default, it is buffered to disk +until the stream is closed in its `close()` method. + +This can make output slow: + +* The execution time for `OutputStream.close()` is proportional to the amount of data +buffered and inversely proportional to the bandwidth. That is `O(data/bandwidth)`. +* The bandwidth is that available from the host to S3: other work in the same +process, server or network at the time of upload may increase the upload time, +hence the duration of the `close()` call. +* If a process uploading data fails before `OutputStream.close()` is called, +all data is lost. +* The disks hosting temporary directories defined in `fs.s3a.buffer.dir` must +have the capacity to store the entire buffered file. + +Put succinctly: the further the process is from the S3 endpoint, or the smaller +the EC-hosted VM is, the longer it will take work to complete. + +This can create problems in application code: + +* Code often assumes that the `close()` call is fast; + the delays can create bottlenecks in operations. +* Very slow uploads sometimes cause applications to time out. (generally, +threads blocking during the upload stop reporting progress, so trigger timeouts) +* Streaming very large amounts of data may consume all disk space before the upload begins. + + +Work to addess this began in Hadoop 2.7 with the `S3AFastOutputStream` + HADOOP-11183 ( https://issues.apache.org/jira/browse/HADOOP-11183 ), and +has continued with ` S3ABlockOutputStream` + HADOOP-13560 ( https://issues.apache.org/jira/browse/HADOOP-13560 ). + + +This adds an alternative output stream, "S3a Fast Upload" which: + +1. Always uploads large files as blocks with the size set by + `fs.s3a.multipart.size`. That is: the threshold at which multipart uploads + begin and the size of each upload are identical. +1. Buffers blocks to disk (default) or in on-heap or off-heap memory. +1. Uploads blocks in parallel in background threads. +1. Begins uploading blocks as soon as the buffered data exceeds this partition + size. +1. When buffering data to disk, uses the directory/directories listed in + `fs.s3a.buffer.dir`. The size of data which can be buffered is limited + to the available disk space. +1. Generates output statistics as metrics on the filesystem, including + statistics of active and pending block uploads. +1. Has the time to `close()` set by the amount of remaning data to upload, rather + than the total size of the file. + +With incremental writes of blocks, "S3A fast upload" offers an upload +time at least as fast as the "classic" mechanism, with significant benefits +on long-lived output streams, and when very large amounts of data are generated. +The in memory buffering mechanims may also offer speedup when running adjacent to +S3 endpoints, as disks are not used for intermediate data storage. + + +```xml +<property> + <name>fs.s3a.fast.upload</name> + <value>true</value> + <description> + Use the incremental block upload mechanism with + the buffering mechanism set in fs.s3a.fast.upload.buffer. + The number of threads performing uploads in the filesystem is defined + by fs.s3a.threads.max; the queue of waiting uploads limited by + fs.s3a.max.total.tasks. + The size of each buffer is set by fs.s3a.multipart.size. + </description> +</property> + +<property> + <name>fs.s3a.fast.upload.buffer</name> + <value>disk</value> + <description> + The buffering mechanism to use when using S3A fast upload + (fs.s3a.fast.upload=true). Values: disk, array, bytebuffer. + This configuration option has no effect if fs.s3a.fast.upload is false. + + "disk" will use the directories listed in fs.s3a.buffer.dir as + the location(s) to save data prior to being uploaded. + + "array" uses arrays in the JVM heap + + "bytebuffer" uses off-heap memory within the JVM. + + Both "array" and "bytebuffer" will consume memory in a single stream up to the number + of blocks set by: + + fs.s3a.multipart.size * fs.s3a.fast.upload.active.blocks. + + If using either of these mechanisms, keep this value low + + The total number of threads performing work across all threads is set by + fs.s3a.threads.max, with fs.s3a.max.total.tasks values setting the number of queued + work items. End diff – ..again, not changing it in either place, as once renames() parallelize, life gets more complex
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user thodemoor commented on a diff in the pull request:

          https://github.com/apache/hadoop/pull/130#discussion_r83419562

          — Diff: hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java —
          @@ -0,0 +1,699 @@
          +/*
          + * Licensed to the Apache Software Foundation (ASF) under one
          + * or more contributor license agreements. See the NOTICE file
          + * distributed with this work for additional information
          + * regarding copyright ownership. The ASF licenses this file
          + * to you under the Apache License, Version 2.0 (the
          + * "License"); you may not use this file except in compliance
          + * with the License. You may obtain a copy of the License at
          + *
          + * http://www.apache.org/licenses/LICENSE-2.0
          + *
          + * Unless required by applicable law or agreed to in writing, software
          + * distributed under the License is distributed on an "AS IS" BASIS,
          + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
          + * See the License for the specific language governing permissions and
          + * limitations under the License.
          + */
          +
          +package org.apache.hadoop.fs.s3a;
          +
          +import java.io.IOException;
          +import java.io.InputStream;
          +import java.io.OutputStream;
          +import java.util.ArrayList;
          +import java.util.List;
          +import java.util.concurrent.Callable;
          +import java.util.concurrent.ExecutionException;
          +import java.util.concurrent.ExecutorService;
          +import java.util.concurrent.TimeUnit;
          +import java.util.concurrent.atomic.AtomicBoolean;
          +
          +import com.amazonaws.AmazonClientException;
          +import com.amazonaws.event.ProgressEvent;
          +import com.amazonaws.event.ProgressEventType;
          +import com.amazonaws.event.ProgressListener;
          +import com.amazonaws.services.s3.model.CompleteMultipartUploadResult;
          +import com.amazonaws.services.s3.model.PartETag;
          +import com.amazonaws.services.s3.model.PutObjectRequest;
          +import com.amazonaws.services.s3.model.PutObjectResult;
          +import com.amazonaws.services.s3.model.UploadPartRequest;
          +import com.google.common.base.Preconditions;
          +import com.google.common.util.concurrent.Futures;
          +import com.google.common.util.concurrent.ListenableFuture;
          +import com.google.common.util.concurrent.ListeningExecutorService;
          +import com.google.common.util.concurrent.MoreExecutors;
          +import org.slf4j.Logger;
          +import org.slf4j.LoggerFactory;
          +
          +import org.apache.hadoop.classification.InterfaceAudience;
          +import org.apache.hadoop.classification.InterfaceStability;
          +import org.apache.hadoop.io.IOUtils;
          +import org.apache.hadoop.io.retry.RetryPolicies;
          +import org.apache.hadoop.io.retry.RetryPolicy;
          +import org.apache.hadoop.util.Progressable;
          +
          +import static org.apache.hadoop.fs.s3a.S3AUtils.*;
          +import static org.apache.hadoop.fs.s3a.Statistic.*;
          +
          +/**
          + * Upload files/parts directly via different buffering mechanisms:
          + * including memory and disk.
          + *
          + * If the stream is closed and no update has started, then the upload
          + * is instead done as a single PUT operation.
          + *
          + * Unstable: statistics and error handling might evolve.
          + */
          +@InterfaceAudience.Private
          +@InterfaceStability.Unstable
          +class S3ABlockOutputStream extends OutputStream {
          +
          + private static final Logger LOG =
          + LoggerFactory.getLogger(S3ABlockOutputStream.class);
          +
          + /** Owner FileSystem. */
          + private final S3AFileSystem fs;
          +
          + /** Object being uploaded. */
          + private final String key;
          +
          + /** Size of all blocks. */
          + private final int blockSize;
          +
          + /** Callback for progress. */
          + private final ProgressListener progressListener;
          + private final ListeningExecutorService executorService;
          +
          + /**
          + * Retry policy for multipart commits; not all AWS SDK versions retry that.
          + */
          + private final RetryPolicy retryPolicy =
          + RetryPolicies.retryUpToMaximumCountWithProportionalSleep(
          + 5,
          + 2000,
          + TimeUnit.MILLISECONDS);
          + /**
          + * Factory for blocks.
          + */
          + private final S3ADataBlocks.BlockFactory blockFactory;
          +
          + /** Preallocated byte buffer for writing single characters. */
          + private final byte[] singleCharWrite = new byte[1];
          +
          + /** Multipart upload details; null means none started. */
          + private MultiPartUpload multiPartUpload;
          +
          + /** Closed flag. */
          + private final AtomicBoolean closed = new AtomicBoolean(false);
          +
          + /** Current data block. Null means none currently active */
          + private S3ADataBlocks.DataBlock activeBlock;
          +
          + /** Count of blocks uploaded. */
          + private long blockCount = 0;
          +
          + /** Statistics to build up. */
          + private final S3AInstrumentation.OutputStreamStatistics statistics;
          +
          + /**
          + * Write operation helper; encapsulation of the filesystem operations.
          + */
          + private final S3AFileSystem.WriteOperationHelper writeOperationHelper;
          +
          + /**
          + * An S3A output stream which uploads partitions in a separate pool of
          + * threads; different

          {@link S3ADataBlocks.BlockFactory}

          + * instances can control where data is buffered.
          + *
          + * @param fs S3AFilesystem
          + * @param key S3 object to work on.
          + * @param executorService the executor service to use to schedule work
          + * @param progress report progress in order to prevent timeouts. If
          + * this object implements

          {@code ProgressListener}

          then it will be
          + * directly wired up to the AWS client, so receive detailed progress
          + * information.
          + * @param blockSize size of a single block.
          + * @param blockFactory factory for creating stream destinations
          + * @param statistics stats for this stream
          + * @param writeOperationHelper state of the write operation.
          + * @throws IOException on any problem
          + */
          + S3ABlockOutputStream(S3AFileSystem fs,
          + String key,
          + ExecutorService executorService,
          + Progressable progress,
          + long blockSize,
          + S3ADataBlocks.BlockFactory blockFactory,
          + S3AInstrumentation.OutputStreamStatistics statistics,
          + S3AFileSystem.WriteOperationHelper writeOperationHelper)
          + throws IOException {
          + this.fs = fs;
          + this.key = key;
          + this.blockFactory = blockFactory;
          + this.blockSize = (int) blockSize;
          + this.statistics = statistics;
          + this.writeOperationHelper = writeOperationHelper;
          + Preconditions.checkArgument(blockSize >= Constants.MULTIPART_MIN_SIZE,
          + "Block size is too small: %d", blockSize);
          + this.executorService = MoreExecutors.listeningDecorator(executorService);
          + this.multiPartUpload = null;
          + this.progressListener = (progress instanceof ProgressListener) ?
          + (ProgressListener) progress
          + : new ProgressableListener(progress);
          + // create that first block. This guarantees that an open + close sequence
          + // writes a 0-byte entry.
          + createBlockIfNeeded();
          + LOG.debug("Initialized S3ABlockOutputStream for {}" +
          + " output to {}", writeOperationHelper, activeBlock);
          + }
          +
          + /**
          + * Demand create a destination block.
          + * @return the active block; null if there isn't one.
          + * @throws IOException on any failure to create
          + */
          + private synchronized S3ADataBlocks.DataBlock createBlockIfNeeded()
          + throws IOException {
          + if (activeBlock == null)

          { + blockCount++; + activeBlock = blockFactory.create(this.blockSize); + }

          + return activeBlock;
          + }
          +
          + /**
          + * Synchronized accessor to the active block.
          + * @return the active block; null if there isn't one.
          + */
          + private synchronized S3ADataBlocks.DataBlock getActiveBlock()

          { + return activeBlock; + }

          +
          + /**
          + * Predicate to query whether or not there is an active block.
          + * @return true if there is an active block.
          + */
          + private synchronized boolean hasActiveBlock()

          { + return activeBlock != null; + }

          +
          + /**
          + * Clear the active block.
          + */
          + private void clearActiveBlock() {
          + LOG.debug("Clearing active block");
          + synchronized (this)

          { + activeBlock = null; + }

          + }
          +
          + /**
          + * Check for the filesystem being open.
          + * @throws IOException if the filesystem is closed.
          + */
          + void checkOpen() throws IOException {
          + if (closed.get())

          { + throw new IOException("Filesystem " + writeOperationHelper + " closed"); + }

          + }
          +
          + /**
          + * The flush operation does not trigger an upload; that awaits
          + * the next block being full. What it does do is call

          {@code flush() }

          + * on the current block, leaving it to choose how to react.
          + * @throws IOException Any IO problem.
          + */
          + @Override
          + public synchronized void flush() throws IOException {
          + checkOpen();
          + S3ADataBlocks.DataBlock dataBlock = getActiveBlock();
          + if (dataBlock != null)

          { + dataBlock.flush(); + }

          + }
          +
          + /**
          + * Writes a byte to the destination. If this causes the buffer to reach
          + * its limit, the actual upload is submitted to the threadpool.
          + * @param b the int of which the lowest byte is written
          + * @throws IOException on any problem
          + */
          + @Override
          + public synchronized void write(int b) throws IOException

          { + singleCharWrite[0] = (byte)b; + write(singleCharWrite, 0, 1); + }

          +
          + /**
          + * Writes a range of bytes from to the memory buffer. If this causes the
          + * buffer to reach its limit, the actual upload is submitted to the
          + * threadpool and the remainder of the array is written to memory
          + * (recursively).
          + * @param source byte array containing
          + * @param offset offset in array where to start
          + * @param len number of bytes to be written
          + * @throws IOException on any problem
          + */
          + @Override
          + public synchronized void write(byte[] source, int offset, int len)
          + throws IOException {
          +
          + S3ADataBlocks.validateWriteArgs(source, offset, len);
          + checkOpen();
          + if (len == 0)

          { + return; + }

          + S3ADataBlocks.DataBlock block = createBlockIfNeeded();
          + int written = block.write(source, offset, len);
          + int remainingCapacity = block.remainingCapacity();
          + if (written < len) {
          + // not everything was written —the block has run out
          + // of capacity
          + // Trigger an upload then process the remainder.
          + LOG.debug("writing more data than block has capacity -triggering upload");
          + uploadCurrentBlock();
          + // tail recursion is mildly expensive, but given buffer sizes must be MB.
          — End diff –

          We can . With the min part size of 5MB you need a 50GB upload to test this. Will take a while vs. AWS. We can test this cheaply, but of course vs our S3-clone, but at least that will test the log @ error.
          @pieterreuse please add this to our testplan

          Show
          githubbot ASF GitHub Bot added a comment - Github user thodemoor commented on a diff in the pull request: https://github.com/apache/hadoop/pull/130#discussion_r83419562 — Diff: hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java — @@ -0,0 +1,699 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.s3a; + +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.Callable; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; + +import com.amazonaws.AmazonClientException; +import com.amazonaws.event.ProgressEvent; +import com.amazonaws.event.ProgressEventType; +import com.amazonaws.event.ProgressListener; +import com.amazonaws.services.s3.model.CompleteMultipartUploadResult; +import com.amazonaws.services.s3.model.PartETag; +import com.amazonaws.services.s3.model.PutObjectRequest; +import com.amazonaws.services.s3.model.PutObjectResult; +import com.amazonaws.services.s3.model.UploadPartRequest; +import com.google.common.base.Preconditions; +import com.google.common.util.concurrent.Futures; +import com.google.common.util.concurrent.ListenableFuture; +import com.google.common.util.concurrent.ListeningExecutorService; +import com.google.common.util.concurrent.MoreExecutors; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.io.IOUtils; +import org.apache.hadoop.io.retry.RetryPolicies; +import org.apache.hadoop.io.retry.RetryPolicy; +import org.apache.hadoop.util.Progressable; + +import static org.apache.hadoop.fs.s3a.S3AUtils.*; +import static org.apache.hadoop.fs.s3a.Statistic.*; + +/** + * Upload files/parts directly via different buffering mechanisms: + * including memory and disk. + * + * If the stream is closed and no update has started, then the upload + * is instead done as a single PUT operation. + * + * Unstable: statistics and error handling might evolve. + */ +@InterfaceAudience.Private +@InterfaceStability.Unstable +class S3ABlockOutputStream extends OutputStream { + + private static final Logger LOG = + LoggerFactory.getLogger(S3ABlockOutputStream.class); + + /** Owner FileSystem. */ + private final S3AFileSystem fs; + + /** Object being uploaded. */ + private final String key; + + /** Size of all blocks. */ + private final int blockSize; + + /** Callback for progress. */ + private final ProgressListener progressListener; + private final ListeningExecutorService executorService; + + /** + * Retry policy for multipart commits; not all AWS SDK versions retry that. + */ + private final RetryPolicy retryPolicy = + RetryPolicies.retryUpToMaximumCountWithProportionalSleep( + 5, + 2000, + TimeUnit.MILLISECONDS); + /** + * Factory for blocks. + */ + private final S3ADataBlocks.BlockFactory blockFactory; + + /** Preallocated byte buffer for writing single characters. */ + private final byte[] singleCharWrite = new byte [1] ; + + /** Multipart upload details; null means none started. */ + private MultiPartUpload multiPartUpload; + + /** Closed flag. */ + private final AtomicBoolean closed = new AtomicBoolean(false); + + /** Current data block. Null means none currently active */ + private S3ADataBlocks.DataBlock activeBlock; + + /** Count of blocks uploaded. */ + private long blockCount = 0; + + /** Statistics to build up. */ + private final S3AInstrumentation.OutputStreamStatistics statistics; + + /** + * Write operation helper; encapsulation of the filesystem operations. + */ + private final S3AFileSystem.WriteOperationHelper writeOperationHelper; + + /** + * An S3A output stream which uploads partitions in a separate pool of + * threads; different {@link S3ADataBlocks.BlockFactory} + * instances can control where data is buffered. + * + * @param fs S3AFilesystem + * @param key S3 object to work on. + * @param executorService the executor service to use to schedule work + * @param progress report progress in order to prevent timeouts. If + * this object implements {@code ProgressListener} then it will be + * directly wired up to the AWS client, so receive detailed progress + * information. + * @param blockSize size of a single block. + * @param blockFactory factory for creating stream destinations + * @param statistics stats for this stream + * @param writeOperationHelper state of the write operation. + * @throws IOException on any problem + */ + S3ABlockOutputStream(S3AFileSystem fs, + String key, + ExecutorService executorService, + Progressable progress, + long blockSize, + S3ADataBlocks.BlockFactory blockFactory, + S3AInstrumentation.OutputStreamStatistics statistics, + S3AFileSystem.WriteOperationHelper writeOperationHelper) + throws IOException { + this.fs = fs; + this.key = key; + this.blockFactory = blockFactory; + this.blockSize = (int) blockSize; + this.statistics = statistics; + this.writeOperationHelper = writeOperationHelper; + Preconditions.checkArgument(blockSize >= Constants.MULTIPART_MIN_SIZE, + "Block size is too small: %d", blockSize); + this.executorService = MoreExecutors.listeningDecorator(executorService); + this.multiPartUpload = null; + this.progressListener = (progress instanceof ProgressListener) ? + (ProgressListener) progress + : new ProgressableListener(progress); + // create that first block. This guarantees that an open + close sequence + // writes a 0-byte entry. + createBlockIfNeeded(); + LOG.debug("Initialized S3ABlockOutputStream for {}" + + " output to {}", writeOperationHelper, activeBlock); + } + + /** + * Demand create a destination block. + * @return the active block; null if there isn't one. + * @throws IOException on any failure to create + */ + private synchronized S3ADataBlocks.DataBlock createBlockIfNeeded() + throws IOException { + if (activeBlock == null) { + blockCount++; + activeBlock = blockFactory.create(this.blockSize); + } + return activeBlock; + } + + /** + * Synchronized accessor to the active block. + * @return the active block; null if there isn't one. + */ + private synchronized S3ADataBlocks.DataBlock getActiveBlock() { + return activeBlock; + } + + /** + * Predicate to query whether or not there is an active block. + * @return true if there is an active block. + */ + private synchronized boolean hasActiveBlock() { + return activeBlock != null; + } + + /** + * Clear the active block. + */ + private void clearActiveBlock() { + LOG.debug("Clearing active block"); + synchronized (this) { + activeBlock = null; + } + } + + /** + * Check for the filesystem being open. + * @throws IOException if the filesystem is closed. + */ + void checkOpen() throws IOException { + if (closed.get()) { + throw new IOException("Filesystem " + writeOperationHelper + " closed"); + } + } + + /** + * The flush operation does not trigger an upload; that awaits + * the next block being full. What it does do is call {@code flush() } + * on the current block, leaving it to choose how to react. + * @throws IOException Any IO problem. + */ + @Override + public synchronized void flush() throws IOException { + checkOpen(); + S3ADataBlocks.DataBlock dataBlock = getActiveBlock(); + if (dataBlock != null) { + dataBlock.flush(); + } + } + + /** + * Writes a byte to the destination. If this causes the buffer to reach + * its limit, the actual upload is submitted to the threadpool. + * @param b the int of which the lowest byte is written + * @throws IOException on any problem + */ + @Override + public synchronized void write(int b) throws IOException { + singleCharWrite[0] = (byte)b; + write(singleCharWrite, 0, 1); + } + + /** + * Writes a range of bytes from to the memory buffer. If this causes the + * buffer to reach its limit, the actual upload is submitted to the + * threadpool and the remainder of the array is written to memory + * (recursively). + * @param source byte array containing + * @param offset offset in array where to start + * @param len number of bytes to be written + * @throws IOException on any problem + */ + @Override + public synchronized void write(byte[] source, int offset, int len) + throws IOException { + + S3ADataBlocks.validateWriteArgs(source, offset, len); + checkOpen(); + if (len == 0) { + return; + } + S3ADataBlocks.DataBlock block = createBlockIfNeeded(); + int written = block.write(source, offset, len); + int remainingCapacity = block.remainingCapacity(); + if (written < len) { + // not everything was written —the block has run out + // of capacity + // Trigger an upload then process the remainder. + LOG.debug("writing more data than block has capacity -triggering upload"); + uploadCurrentBlock(); + // tail recursion is mildly expensive, but given buffer sizes must be MB. — End diff – We can . With the min part size of 5MB you need a 50GB upload to test this. Will take a while vs. AWS. We can test this cheaply, but of course vs our S3-clone, but at least that will test the log @ error. @pieterreuse please add this to our testplan
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user steveloughran commented on a diff in the pull request:

          https://github.com/apache/hadoop/pull/130#discussion_r83419925

          — Diff: hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md —
          @@ -881,40 +881,362 @@ Seoul
          If the wrong endpoint is used, the request may fail. This may be reported as a 301/redirect error,
          or as a 400 Bad Request.

                1. S3AFastOutputStream
          • *Warning: NEW in hadoop 2.7. UNSTABLE, EXPERIMENTAL: use at own risk*
          • <property>
          • <name>fs.s3a.fast.upload</name>
          • <value>false</value>
          • <description>Upload directly from memory instead of buffering to
          • disk first. Memory usage and parallelism can be controlled as up to
          • fs.s3a.multipart.size memory is consumed for each (part)upload actively
          • uploading (fs.s3a.threads.max) or queueing (fs.s3a.max.total.tasks)</description>
          • </property>
          • <property>
          • <name>fs.s3a.fast.buffer.size</name>
          • <value>1048576</value>
          • <description>Size (in bytes) of initial memory buffer allocated for an
          • upload. No effect if fs.s3a.fast.upload is false.</description>
          • </property>
            +### <a name="s3a_fast_upload"></a>Stabilizing: S3A Fast Upload
            +
            +
            +*New in Hadoop 2.7; significantly enhanced in Hadoop 2.9*
            +
            +
            +Because of the nature of the S3 object store, data written to an S3A `OutputStream`
            +is not written incrementally —instead, by default, it is buffered to disk
            +until the stream is closed in its `close()` method.
            +
            +This can make output slow:
            +
            +* The execution time for `OutputStream.close()` is proportional to the amount of data
            +buffered and inversely proportional to the bandwidth. That is `O(data/bandwidth)`.
            +* The bandwidth is that available from the host to S3: other work in the same
            +process, server or network at the time of upload may increase the upload time,
            +hence the duration of the `close()` call.
            +* If a process uploading data fails before `OutputStream.close()` is called,
            +all data is lost.
            +* The disks hosting temporary directories defined in `fs.s3a.buffer.dir` must
            +have the capacity to store the entire buffered file.
            +
            +Put succinctly: the further the process is from the S3 endpoint, or the smaller
            +the EC-hosted VM is, the longer it will take work to complete.
            +
            +This can create problems in application code:
            +
            +* Code often assumes that the `close()` call is fast;
            + the delays can create bottlenecks in operations.
            +* Very slow uploads sometimes cause applications to time out. (generally,
            +threads blocking during the upload stop reporting progress, so trigger timeouts)
            +* Streaming very large amounts of data may consume all disk space before the upload begins.
            +
            +
            +Work to addess this began in Hadoop 2.7 with the `S3AFastOutputStream`
            +HADOOP-11183(https://issues.apache.org/jira/browse/HADOOP-11183), and
            +has continued with ` S3ABlockOutputStream`
            +HADOOP-13560(https://issues.apache.org/jira/browse/HADOOP-13560).
            +
            +
            +This adds an alternative output stream, "S3a Fast Upload" which:
            +
            +1. Always uploads large files as blocks with the size set by
            + `fs.s3a.multipart.size`. That is: the threshold at which multipart uploads
            + begin and the size of each upload are identical.
            +1. Buffers blocks to disk (default) or in on-heap or off-heap memory.
            +1. Uploads blocks in parallel in background threads.
            +1. Begins uploading blocks as soon as the buffered data exceeds this partition
            + size.
            +1. When buffering data to disk, uses the directory/directories listed in
            + `fs.s3a.buffer.dir`. The size of data which can be buffered is limited
            + to the available disk space.
            +1. Generates output statistics as metrics on the filesystem, including
            + statistics of active and pending block uploads.
            +1. Has the time to `close()` set by the amount of remaning data to upload, rather
            + than the total size of the file.
            +
            +With incremental writes of blocks, "S3A fast upload" offers an upload
            +time at least as fast as the "classic" mechanism, with significant benefits
            +on long-lived output streams, and when very large amounts of data are generated.
            +The in memory buffering mechanims may also offer speedup when running adjacent to
            +S3 endpoints, as disks are not used for intermediate data storage.
            +
            +
            +```xml
            +<property>
            + <name>fs.s3a.fast.upload</name>
            + <value>true</value>
            + <description>
            + Use the incremental block upload mechanism with
            + the buffering mechanism set in fs.s3a.fast.upload.buffer.
            + The number of threads performing uploads in the filesystem is defined
            + by fs.s3a.threads.max; the queue of waiting uploads limited by
            + fs.s3a.max.total.tasks.
            + The size of each buffer is set by fs.s3a.multipart.size.
            + </description>
            +</property>
            +
            +<property>
            + <name>fs.s3a.fast.upload.buffer</name>
            + <value>disk</value>
            + <description>
            + The buffering mechanism to use when using S3A fast upload
            + (fs.s3a.fast.upload=true). Values: disk, array, bytebuffer.
            + This configuration option has no effect if fs.s3a.fast.upload is false.
            +
            + "disk" will use the directories listed in fs.s3a.buffer.dir as
            + the location(s) to save data prior to being uploaded.
            +
            + "array" uses arrays in the JVM heap
            +
            + "bytebuffer" uses off-heap memory within the JVM.
            +
            + Both "array" and "bytebuffer" will consume memory in a single stream up to the number
            + of blocks set by:
            +
            + fs.s3a.multipart.size * fs.s3a.fast.upload.active.blocks.
            +
            + If using either of these mechanisms, keep this value low
            +
            + The total number of threads performing work across all threads is set by
            + fs.s3a.threads.max, with fs.s3a.max.total.tasks values setting the number of queued
            + work items.
            + </description>
            +</property>
            +
            +<property>
            + <name>fs.s3a.multipart.size</name>
            + <value>104857600</value>
            + <description>
            + How big (in bytes) to split upload or copy operations up into.
            + </description>
            +</property>
            +
            +<property>
            + <name>fs.s3a.fast.upload.active.blocks</name>
            + <value>8</value>
            + <description>
            + Maximum Number of blocks a single output stream can have
            + active (uploading, or queued to the central FileSystem
            + instance's pool of queued operations.
            +
            + This stops a single stream overloading the shared thread pool.
            + </description>
            +</property>
            +```
            +
            +*Notes*
            +
            +* If the amount of data written to a stream is below that set in `fs.s3a.multipart.size`,
            +the upload is performed in the `OutputStream.close()` operation —as with
            +the original output stream.
            +
            +* The published Hadoop metrics monitor include live queue length and
            +upload operation counts, so identifying when there is a backlog of work/
            +a mismatch between data generation rates and network bandwidth. Per-stream
            +statistics can also be logged by calling `toString()` on the current stream.
            +
            +* Incremental writes are not visible; the object can only be listed
            +or read when the multipart operation completes in the `close()` call, which
            +will block until the upload is completed.
            +
            +
            +#### <a name="s3a_fast_upload_disk"></a>Fast Upload with Disk Buffers `fs.s3a.fast.upload.buffer=disk`
            +
            +When `fs.s3a.fast.upload.buffer` is set to `disk`, all data is buffered
            +to local hard disks prior to upload. This minimizes the amount of memory
            +consumed, and so eliminates heap size as the limiting factor in queued uploads
            +—exactly as the original "direct to disk" buffering used when
            +`fs.s3a.fast.upload=false`.
            +
            +
            +```xml
            +<property>
            + <name>fs.s3a.fast.upload</name>
            + <value>true</value>
            +</property>
            +
            +<property>
            + <name>fs.s3a.fast.upload.buffer</name>
            + <value>disk</value>
            +</property>
            +
            +```
            +
            +
            +#### <a name="s3a_fast_upload_bytebuffer"></a>Fast Upload with ByteBuffers: `fs.s3a.fast.upload.buffer=bytebuffer`
            +
            +When `fs.s3a.fast.upload.buffer` is set to `bytebuffer`, all data is buffered
            +in "Direct" ByteBuffers prior to upload. This may be faster than buffering to disk,
            +and, if disk space is small (for example, tiny EC2 VMs), there may not
            +be much disk space to buffer with.
            +
            +The ByteBuffers are created in the memory of the JVM, but not in the Java Heap itself.
            +The amount of data which can be buffered is
            +limited by the Java runtime, the operating system, and, for YARN applications,
            +the amount of memory requested for each container.
            +
            +The slower the write bandwidth to S3, the greater the risk of running out
            +of memory.
            +
            +
            +```xml
            +<property>
            + <name>fs.s3a.fast.upload</name>
            + <value>true</value>
            +</property>
            +
            +<property>
            + <name>fs.s3a.fast.upload.buffer</name>
            + <value>bytebuffer</value>
            +</property>
            +```
            +
            +#### <a name="s3a_fast_upload_array"></a>Fast Upload with Arrays: `fs.s3a.fast.upload.buffer=array`
            +
            +When `fs.s3a.fast.upload.buffer` is set to `array`, all data is buffered
            +in byte arrays in the JVM's heap prior to upload.
            +This may be faster than buffering to disk.
            +
            +This `array` option is similar to the in-memory-only stream offered in
            +Hadoop 2.7 with `fs.s3a.fast.upload=true`
            +
            +The amount of data which can be buffered is limited by the available
            +size of the JVM heap heap. The slower the write bandwidth to S3, the greater
            +the risk of heap overflows.
              • End diff –

          adding link to the s3a_fast_upload_thread_tuning section

          Show
          githubbot ASF GitHub Bot added a comment - Github user steveloughran commented on a diff in the pull request: https://github.com/apache/hadoop/pull/130#discussion_r83419925 — Diff: hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md — @@ -881,40 +881,362 @@ Seoul If the wrong endpoint is used, the request may fail. This may be reported as a 301/redirect error, or as a 400 Bad Request. S3AFastOutputStream * Warning: NEW in hadoop 2.7. UNSTABLE, EXPERIMENTAL: use at own risk * <property> <name>fs.s3a.fast.upload</name> <value>false</value> <description>Upload directly from memory instead of buffering to disk first. Memory usage and parallelism can be controlled as up to fs.s3a.multipart.size memory is consumed for each (part)upload actively uploading (fs.s3a.threads.max) or queueing (fs.s3a.max.total.tasks)</description> </property> <property> <name>fs.s3a.fast.buffer.size</name> <value>1048576</value> <description>Size (in bytes) of initial memory buffer allocated for an upload. No effect if fs.s3a.fast.upload is false.</description> </property> +### <a name="s3a_fast_upload"></a>Stabilizing: S3A Fast Upload + + +* New in Hadoop 2.7; significantly enhanced in Hadoop 2.9 * + + +Because of the nature of the S3 object store, data written to an S3A `OutputStream` +is not written incrementally —instead, by default, it is buffered to disk +until the stream is closed in its `close()` method. + +This can make output slow: + +* The execution time for `OutputStream.close()` is proportional to the amount of data +buffered and inversely proportional to the bandwidth. That is `O(data/bandwidth)`. +* The bandwidth is that available from the host to S3: other work in the same +process, server or network at the time of upload may increase the upload time, +hence the duration of the `close()` call. +* If a process uploading data fails before `OutputStream.close()` is called, +all data is lost. +* The disks hosting temporary directories defined in `fs.s3a.buffer.dir` must +have the capacity to store the entire buffered file. + +Put succinctly: the further the process is from the S3 endpoint, or the smaller +the EC-hosted VM is, the longer it will take work to complete. + +This can create problems in application code: + +* Code often assumes that the `close()` call is fast; + the delays can create bottlenecks in operations. +* Very slow uploads sometimes cause applications to time out. (generally, +threads blocking during the upload stop reporting progress, so trigger timeouts) +* Streaming very large amounts of data may consume all disk space before the upload begins. + + +Work to addess this began in Hadoop 2.7 with the `S3AFastOutputStream` + HADOOP-11183 ( https://issues.apache.org/jira/browse/HADOOP-11183 ), and +has continued with ` S3ABlockOutputStream` + HADOOP-13560 ( https://issues.apache.org/jira/browse/HADOOP-13560 ). + + +This adds an alternative output stream, "S3a Fast Upload" which: + +1. Always uploads large files as blocks with the size set by + `fs.s3a.multipart.size`. That is: the threshold at which multipart uploads + begin and the size of each upload are identical. +1. Buffers blocks to disk (default) or in on-heap or off-heap memory. +1. Uploads blocks in parallel in background threads. +1. Begins uploading blocks as soon as the buffered data exceeds this partition + size. +1. When buffering data to disk, uses the directory/directories listed in + `fs.s3a.buffer.dir`. The size of data which can be buffered is limited + to the available disk space. +1. Generates output statistics as metrics on the filesystem, including + statistics of active and pending block uploads. +1. Has the time to `close()` set by the amount of remaning data to upload, rather + than the total size of the file. + +With incremental writes of blocks, "S3A fast upload" offers an upload +time at least as fast as the "classic" mechanism, with significant benefits +on long-lived output streams, and when very large amounts of data are generated. +The in memory buffering mechanims may also offer speedup when running adjacent to +S3 endpoints, as disks are not used for intermediate data storage. + + +```xml +<property> + <name>fs.s3a.fast.upload</name> + <value>true</value> + <description> + Use the incremental block upload mechanism with + the buffering mechanism set in fs.s3a.fast.upload.buffer. + The number of threads performing uploads in the filesystem is defined + by fs.s3a.threads.max; the queue of waiting uploads limited by + fs.s3a.max.total.tasks. + The size of each buffer is set by fs.s3a.multipart.size. + </description> +</property> + +<property> + <name>fs.s3a.fast.upload.buffer</name> + <value>disk</value> + <description> + The buffering mechanism to use when using S3A fast upload + (fs.s3a.fast.upload=true). Values: disk, array, bytebuffer. + This configuration option has no effect if fs.s3a.fast.upload is false. + + "disk" will use the directories listed in fs.s3a.buffer.dir as + the location(s) to save data prior to being uploaded. + + "array" uses arrays in the JVM heap + + "bytebuffer" uses off-heap memory within the JVM. + + Both "array" and "bytebuffer" will consume memory in a single stream up to the number + of blocks set by: + + fs.s3a.multipart.size * fs.s3a.fast.upload.active.blocks. + + If using either of these mechanisms, keep this value low + + The total number of threads performing work across all threads is set by + fs.s3a.threads.max, with fs.s3a.max.total.tasks values setting the number of queued + work items. + </description> +</property> + +<property> + <name>fs.s3a.multipart.size</name> + <value>104857600</value> + <description> + How big (in bytes) to split upload or copy operations up into. + </description> +</property> + +<property> + <name>fs.s3a.fast.upload.active.blocks</name> + <value>8</value> + <description> + Maximum Number of blocks a single output stream can have + active (uploading, or queued to the central FileSystem + instance's pool of queued operations. + + This stops a single stream overloading the shared thread pool. + </description> +</property> +``` + +* Notes * + +* If the amount of data written to a stream is below that set in `fs.s3a.multipart.size`, +the upload is performed in the `OutputStream.close()` operation —as with +the original output stream. + +* The published Hadoop metrics monitor include live queue length and +upload operation counts, so identifying when there is a backlog of work/ +a mismatch between data generation rates and network bandwidth. Per-stream +statistics can also be logged by calling `toString()` on the current stream. + +* Incremental writes are not visible; the object can only be listed +or read when the multipart operation completes in the `close()` call, which +will block until the upload is completed. + + +#### <a name="s3a_fast_upload_disk"></a>Fast Upload with Disk Buffers `fs.s3a.fast.upload.buffer=disk` + +When `fs.s3a.fast.upload.buffer` is set to `disk`, all data is buffered +to local hard disks prior to upload. This minimizes the amount of memory +consumed, and so eliminates heap size as the limiting factor in queued uploads +—exactly as the original "direct to disk" buffering used when +`fs.s3a.fast.upload=false`. + + +```xml +<property> + <name>fs.s3a.fast.upload</name> + <value>true</value> +</property> + +<property> + <name>fs.s3a.fast.upload.buffer</name> + <value>disk</value> +</property> + +``` + + +#### <a name="s3a_fast_upload_bytebuffer"></a>Fast Upload with ByteBuffers: `fs.s3a.fast.upload.buffer=bytebuffer` + +When `fs.s3a.fast.upload.buffer` is set to `bytebuffer`, all data is buffered +in "Direct" ByteBuffers prior to upload. This may be faster than buffering to disk, +and, if disk space is small (for example, tiny EC2 VMs), there may not +be much disk space to buffer with. + +The ByteBuffers are created in the memory of the JVM, but not in the Java Heap itself. +The amount of data which can be buffered is +limited by the Java runtime, the operating system, and, for YARN applications, +the amount of memory requested for each container. + +The slower the write bandwidth to S3, the greater the risk of running out +of memory. + + +```xml +<property> + <name>fs.s3a.fast.upload</name> + <value>true</value> +</property> + +<property> + <name>fs.s3a.fast.upload.buffer</name> + <value>bytebuffer</value> +</property> +``` + +#### <a name="s3a_fast_upload_array"></a>Fast Upload with Arrays: `fs.s3a.fast.upload.buffer=array` + +When `fs.s3a.fast.upload.buffer` is set to `array`, all data is buffered +in byte arrays in the JVM's heap prior to upload. +This may be faster than buffering to disk. + +This `array` option is similar to the in-memory-only stream offered in +Hadoop 2.7 with `fs.s3a.fast.upload=true` + +The amount of data which can be buffered is limited by the available +size of the JVM heap heap. The slower the write bandwidth to S3, the greater +the risk of heap overflows. End diff – adding link to the s3a_fast_upload_thread_tuning section
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user steveloughran commented on a diff in the pull request:

          https://github.com/apache/hadoop/pull/130#discussion_r83420236

          — Diff: hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md —
          @@ -881,40 +881,362 @@ Seoul
          If the wrong endpoint is used, the request may fail. This may be reported as a 301/redirect error,
          or as a 400 Bad Request.

                1. S3AFastOutputStream
          • *Warning: NEW in hadoop 2.7. UNSTABLE, EXPERIMENTAL: use at own risk*
          • <property>
          • <name>fs.s3a.fast.upload</name>
          • <value>false</value>
          • <description>Upload directly from memory instead of buffering to
          • disk first. Memory usage and parallelism can be controlled as up to
          • fs.s3a.multipart.size memory is consumed for each (part)upload actively
          • uploading (fs.s3a.threads.max) or queueing (fs.s3a.max.total.tasks)</description>
          • </property>
          • <property>
          • <name>fs.s3a.fast.buffer.size</name>
          • <value>1048576</value>
          • <description>Size (in bytes) of initial memory buffer allocated for an
          • upload. No effect if fs.s3a.fast.upload is false.</description>
          • </property>
            +### <a name="s3a_fast_upload"></a>Stabilizing: S3A Fast Upload
            +
            +
            +*New in Hadoop 2.7; significantly enhanced in Hadoop 2.9*
            +
            +
            +Because of the nature of the S3 object store, data written to an S3A `OutputStream`
            +is not written incrementally —instead, by default, it is buffered to disk
            +until the stream is closed in its `close()` method.
            +
            +This can make output slow:
            +
            +* The execution time for `OutputStream.close()` is proportional to the amount of data
            +buffered and inversely proportional to the bandwidth. That is `O(data/bandwidth)`.
            +* The bandwidth is that available from the host to S3: other work in the same
            +process, server or network at the time of upload may increase the upload time,
            +hence the duration of the `close()` call.
            +* If a process uploading data fails before `OutputStream.close()` is called,
            +all data is lost.
            +* The disks hosting temporary directories defined in `fs.s3a.buffer.dir` must
            +have the capacity to store the entire buffered file.
            +
            +Put succinctly: the further the process is from the S3 endpoint, or the smaller
            +the EC-hosted VM is, the longer it will take work to complete.
            +
            +This can create problems in application code:
            +
            +* Code often assumes that the `close()` call is fast;
            + the delays can create bottlenecks in operations.
            +* Very slow uploads sometimes cause applications to time out. (generally,
            +threads blocking during the upload stop reporting progress, so trigger timeouts)
            +* Streaming very large amounts of data may consume all disk space before the upload begins.
            +
            +
            +Work to addess this began in Hadoop 2.7 with the `S3AFastOutputStream`
            +HADOOP-11183(https://issues.apache.org/jira/browse/HADOOP-11183), and
            +has continued with ` S3ABlockOutputStream`
            +HADOOP-13560(https://issues.apache.org/jira/browse/HADOOP-13560).
            +
            +
            +This adds an alternative output stream, "S3a Fast Upload" which:
            +
            +1. Always uploads large files as blocks with the size set by
            + `fs.s3a.multipart.size`. That is: the threshold at which multipart uploads
            + begin and the size of each upload are identical.
            +1. Buffers blocks to disk (default) or in on-heap or off-heap memory.
            +1. Uploads blocks in parallel in background threads.
            +1. Begins uploading blocks as soon as the buffered data exceeds this partition
            + size.
            +1. When buffering data to disk, uses the directory/directories listed in
            + `fs.s3a.buffer.dir`. The size of data which can be buffered is limited
            + to the available disk space.
            +1. Generates output statistics as metrics on the filesystem, including
            + statistics of active and pending block uploads.
            +1. Has the time to `close()` set by the amount of remaning data to upload, rather
            + than the total size of the file.
            +
            +With incremental writes of blocks, "S3A fast upload" offers an upload
            +time at least as fast as the "classic" mechanism, with significant benefits
            +on long-lived output streams, and when very large amounts of data are generated.
            +The in memory buffering mechanims may also offer speedup when running adjacent to
            +S3 endpoints, as disks are not used for intermediate data storage.
            +
            +
            +```xml
            +<property>
            + <name>fs.s3a.fast.upload</name>
            + <value>true</value>
            + <description>
            + Use the incremental block upload mechanism with
            + the buffering mechanism set in fs.s3a.fast.upload.buffer.
            + The number of threads performing uploads in the filesystem is defined
            + by fs.s3a.threads.max; the queue of waiting uploads limited by
            + fs.s3a.max.total.tasks.
            + The size of each buffer is set by fs.s3a.multipart.size.
            + </description>
            +</property>
            +
            +<property>
            + <name>fs.s3a.fast.upload.buffer</name>
            + <value>disk</value>
            + <description>
            + The buffering mechanism to use when using S3A fast upload
            + (fs.s3a.fast.upload=true). Values: disk, array, bytebuffer.
            + This configuration option has no effect if fs.s3a.fast.upload is false.
            +
            + "disk" will use the directories listed in fs.s3a.buffer.dir as
            + the location(s) to save data prior to being uploaded.
            +
            + "array" uses arrays in the JVM heap
            +
            + "bytebuffer" uses off-heap memory within the JVM.
            +
            + Both "array" and "bytebuffer" will consume memory in a single stream up to the number
            + of blocks set by:
            +
            + fs.s3a.multipart.size * fs.s3a.fast.upload.active.blocks.
            +
            + If using either of these mechanisms, keep this value low
            +
            + The total number of threads performing work across all threads is set by
            + fs.s3a.threads.max, with fs.s3a.max.total.tasks values setting the number of queued
            + work items.
            + </description>
            +</property>
            +
            +<property>
            + <name>fs.s3a.multipart.size</name>
            + <value>104857600</value>
            + <description>
            + How big (in bytes) to split upload or copy operations up into.
            + </description>
            +</property>
            +
            +<property>
            + <name>fs.s3a.fast.upload.active.blocks</name>
            + <value>8</value>
            + <description>
            + Maximum Number of blocks a single output stream can have
            + active (uploading, or queued to the central FileSystem
            + instance's pool of queued operations.
            +
            + This stops a single stream overloading the shared thread pool.
            + </description>
            +</property>
            +```
            +
            +*Notes*
            +
            +* If the amount of data written to a stream is below that set in `fs.s3a.multipart.size`,
            +the upload is performed in the `OutputStream.close()` operation —as with
            +the original output stream.
            +
            +* The published Hadoop metrics monitor include live queue length and
            +upload operation counts, so identifying when there is a backlog of work/
            +a mismatch between data generation rates and network bandwidth. Per-stream
            +statistics can also be logged by calling `toString()` on the current stream.
            +
            +* Incremental writes are not visible; the object can only be listed
            +or read when the multipart operation completes in the `close()` call, which
            +will block until the upload is completed.
            +
            +
            +#### <a name="s3a_fast_upload_disk"></a>Fast Upload with Disk Buffers `fs.s3a.fast.upload.buffer=disk`
            +
            +When `fs.s3a.fast.upload.buffer` is set to `disk`, all data is buffered
            +to local hard disks prior to upload. This minimizes the amount of memory
            +consumed, and so eliminates heap size as the limiting factor in queued uploads
            +—exactly as the original "direct to disk" buffering used when
            +`fs.s3a.fast.upload=false`.
            +
            +
            +```xml
            +<property>
            + <name>fs.s3a.fast.upload</name>
            + <value>true</value>
            +</property>
            +
            +<property>
            + <name>fs.s3a.fast.upload.buffer</name>
            + <value>disk</value>
            +</property>
            +
            +```
            +
            +
            +#### <a name="s3a_fast_upload_bytebuffer"></a>Fast Upload with ByteBuffers: `fs.s3a.fast.upload.buffer=bytebuffer`
            +
            +When `fs.s3a.fast.upload.buffer` is set to `bytebuffer`, all data is buffered
            +in "Direct" ByteBuffers prior to upload. This may be faster than buffering to disk,
            +and, if disk space is small (for example, tiny EC2 VMs), there may not
            +be much disk space to buffer with.
            +
            +The ByteBuffers are created in the memory of the JVM, but not in the Java Heap itself.
            +The amount of data which can be buffered is
            +limited by the Java runtime, the operating system, and, for YARN applications,
            +the amount of memory requested for each container.
            +
            +The slower the write bandwidth to S3, the greater the risk of running out
            +of memory.
            +
            +
            +```xml
            +<property>
            + <name>fs.s3a.fast.upload</name>
            + <value>true</value>
            +</property>
            +
            +<property>
            + <name>fs.s3a.fast.upload.buffer</name>
            + <value>bytebuffer</value>
            +</property>
            +```
            +
            +#### <a name="s3a_fast_upload_array"></a>Fast Upload with Arrays: `fs.s3a.fast.upload.buffer=array`
            +
            +When `fs.s3a.fast.upload.buffer` is set to `array`, all data is buffered
            +in byte arrays in the JVM's heap prior to upload.
            +This may be faster than buffering to disk.
            +
            +This `array` option is similar to the in-memory-only stream offered in
            +Hadoop 2.7 with `fs.s3a.fast.upload=true`
            +
            +The amount of data which can be buffered is limited by the available
            +size of the JVM heap heap. The slower the write bandwidth to S3, the greater
            +the risk of heap overflows.
            +
            +```xml
            +<property>
            + <name>fs.s3a.fast.upload</name>
            + <value>true</value>
            +</property>
            +
            +<property>
            + <name>fs.s3a.fast.upload.buffer</name>
            + <value>array</value>
            +</property>
            +
            +```
            +#### <a name="s3a_fast_upload_thread_tuning"></a>S3A Fast Upload Thread Tuning
            +
            +Both the [Array](#s3a_fast_upload_array) and [Byte buffer](#s3a_fast_upload_bytebuffer)
            +buffer mechanisms can consume very large amounts of memory, on-heap or
            +off-heap respectively. The [disk buffer](#s3a_fast_upload_disk) mechanism
            +does not use much memory up, but will consume hard disk capacity.
            +
            +If there are many output streams being written to in a single process, the
            +amount of memory or disk used is the multiple of all stream's active memory/disk use.
            +
            +Careful tuning may be needed to reduce the risk of running out memory, especially
            +if the data is buffered in memory.
            +
            +There are a number parameters which can be tuned:
            +
            +1. The total number of threads available in the filesystem for data
            +uploads or any other queued filesystem operation. This is set in
            +`fs.s3a.threads.max`
            +
            +1. The number of operations which can be queued for execution:, *awaiting
            +a thread*: `fs.s3a.max.total.tasks`
            +
            +1. The number of blocks which a single output stream can have active,
            +that is: being uploaded by a thread, or queued in the filesystem thread queue:
            +`fs.s3a.fast.upload.active.blocks`
            +
            +1. How long an idle thread can stay in the thread pool before it is retired: `fs.s3a.threads.keepalivetime`
            +
            +
            +When the maximum allowed number of active blocks of a single stream is reached,
            +no more blocks can be up