From ce164df60d6345774bcdc175a1941778bafb1d57 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Tue, 30 Aug 2016 16:48:14 +0100 Subject: [PATCH 01/20] HADOOP-13560 adding test for create/copy 5GB files --- .../hadoop/fs/s3a/S3ATestConstants.java | 12 ++ .../fs/s3a/scale/ITestS3ADeleteManyFiles.java | 19 +- .../fs/s3a/scale/ITestS3AHugeFileCreate.java | 184 ++++++++++++++++++ .../hadoop/fs/s3a/scale/S3AScaleTestBase.java | 4 + 4 files changed, 204 insertions(+), 15 deletions(-) create mode 100644 hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AHugeFileCreate.java diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestConstants.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestConstants.java index 6a4e68c9132f8..2be3f429081a9 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestConstants.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestConstants.java @@ -73,4 +73,16 @@ public interface S3ATestConstants { * Run the encryption tests? */ String KEY_ENCRYPTION_TESTS = TEST_FS_S3A + "encryption.enabled"; + + /** + * Name of the property to define the file size for the huge file + * tests: {@value}. Measured in MB. + */ + String KEY_HUGE_FILESIZE = S3A_SCALE_TEST + "huge.filesize"; + + /** + * The default huge size is small —full 5GB+ scale tests are something + * to run in long test runs on EC2 VMs. {@value}. + */ + long DEFAULT_HUGE_FILESIZE = 10L; } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3ADeleteManyFiles.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3ADeleteManyFiles.java index 208c491334f56..4e1a734e564b9 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3ADeleteManyFiles.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3ADeleteManyFiles.java @@ -116,20 +116,9 @@ public Boolean call() throws IOException { @Test public void testOpenCreate() throws IOException { - Path dir = new Path("/tests3a"); - ContractTestUtils.createAndVerifyFile(fs, dir, 1024); - ContractTestUtils.createAndVerifyFile(fs, dir, 5 * 1024 * 1024); - ContractTestUtils.createAndVerifyFile(fs, dir, 20 * 1024 * 1024); - - - /* - Enable to test the multipart upload - try { - ContractTestUtils.createAndVerifyFile(fs, dir, - (long)6 * 1024 * 1024 * 1024); - } catch (IOException e) { - fail(e.getMessage()); - } - */ + final Path scaleTestDir = getTestPath(); + final Path srcDir = new Path(scaleTestDir, "opencreate"); + ContractTestUtils.createAndVerifyFile(fs, srcDir, 1024); + ContractTestUtils.createAndVerifyFile(fs, srcDir, 50 * 1024); } } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AHugeFileCreate.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AHugeFileCreate.java new file mode 100644 index 0000000000000..08f1964eaf912 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AHugeFileCreate.java @@ -0,0 +1,184 @@ +/* + * 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.scale; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.contract.ContractTestUtils; +import org.apache.hadoop.fs.s3a.Constants; +import org.apache.hadoop.fs.s3a.S3AFileStatus; +import org.junit.Assume; +import org.junit.FixMethodOrder; +import org.junit.Test; +import org.junit.runners.MethodSorters; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.FileNotFoundException; +import java.io.IOException; + +import static org.apache.hadoop.fs.contract.ContractTestUtils.toHuman; +import static org.apache.hadoop.fs.s3a.Constants.MIN_MULTIPART_THRESHOLD; +import static org.apache.hadoop.fs.s3a.Constants.SOCKET_RECV_BUFFER; +import static org.apache.hadoop.fs.s3a.Constants.SOCKET_SEND_BUFFER; + +/** + * Scale test which creates a huge file. + * + * Important: the order in which these tests execute is fixed to + * alphabetical order. Test cases are numbered {@code test_123_} to impose + * an ordering based on the numbers. + * + * Having this ordering allows the tests to assume that the huge file + * exists. Even so: they should all have an assumes() check at the start, + * in case an individual test is executed. + */ +@FixMethodOrder(MethodSorters.NAME_ASCENDING) +public class ITestS3AHugeFileCreate extends S3AScaleTestBase { + private static final Logger LOG = LoggerFactory.getLogger( + ITestS3AHugeFileCreate.class); + private Path scaleTestDir; + private Path hugefile; + private Path hugefileRenamed; + + public static final int BLOCKSIZE = 64 * 1024; + + @Override + public void setUp() throws Exception { + super.setUp(); + final Path testPath = getTestPath(); + scaleTestDir = new Path(testPath, "scale"); + hugefile = new Path(scaleTestDir, "hugefile"); + hugefileRenamed = new Path(scaleTestDir, "hugefileRenamed"); + } + + @Override + public void tearDown() throws Exception { + // do nothing + } + + @Override + protected Configuration createConfiguration() { + Configuration configuration = super.createConfiguration(); + configuration.setBoolean(Constants.FAST_UPLOAD, true); + configuration.setLong(MIN_MULTIPART_THRESHOLD, 10 * _1MB); + configuration.setLong(SOCKET_SEND_BUFFER, BLOCKSIZE); + configuration.setLong(SOCKET_RECV_BUFFER, BLOCKSIZE); + return configuration; + } + + @Test + public void test_001_CreateHugeFile() throws IOException { + long mb = getConf().getLong(KEY_HUGE_FILESIZE, DEFAULT_HUGE_FILESIZE); + long filesize = _1MB * mb; + + describe("Creating file %s of size %d MB", hugefile, mb); + try { + long actualSize = fs.getFileStatus(hugefile).getLen(); + Assume.assumeTrue("File of desired size already exists; skipping", + actualSize != filesize); + } catch (FileNotFoundException e) { + } + byte[] data = new byte[BLOCKSIZE]; + for (int i = 0; i < BLOCKSIZE; i++) { + data[i] = (byte)(i % 256); + } + + assertEquals ( + "File size set in " + KEY_HUGE_FILESIZE+ " = " + filesize + +" is not a multiple of " + BLOCKSIZE, + 0, filesize % BLOCKSIZE); + long blocks = filesize / BLOCKSIZE; + long blocksPerMB = _1MB / BLOCKSIZE; + + ContractTestUtils.NanoTimer timer = new ContractTestUtils.NanoTimer(); + try(FSDataOutputStream out = fs.create(hugefile, true)) { + for (long block = 0; block < blocks; block++) { + out.write(data); + if (block > 0 && blocksPerMB % block == 0) { + LOG.info("."); + } + } + } + timer.end("Time to write %d MB in blocks of %d", mb, + BLOCKSIZE); + LOG.info("Time per MB to write = {} nS", toHuman(timer.duration() / mb)); + logFSState(); + S3AFileStatus status = fs.getFileStatus(hugefile); + assertEquals("File size in " + status, filesize, status.getLen()); + } + + void assumeHugeFileExists() throws IOException { + Assume.assumeTrue("No file " + hugefile, fs.exists(hugefile)); + } + + @Test + public void test_050_readHugeFile() throws Throwable { + assumeHugeFileExists(); + describe("Reading %s", hugefile); + S3AFileStatus status = fs.getFileStatus(hugefile); + long filesize = status.getLen(); + long blocks = filesize / BLOCKSIZE; + byte[] data = new byte[BLOCKSIZE]; + + ContractTestUtils.NanoTimer timer = new ContractTestUtils.NanoTimer(); + try (FSDataInputStream in = fs.open(hugefile, BLOCKSIZE)) { + for (long block = 0; block < blocks; block++) { + in.readFully(data); + } + } + + long mb = Math.max(filesize / _1MB, 1); + timer.end("Time to read file of %d MB ", mb); + LOG.info("Time per MB to read = {} nS", toHuman(timer.duration() / mb)); + logFSState(); + } + + private void logFSState() { + LOG.info("File System state after operation; {}", fs); + } + + @Test + public void test_100_renameHugeFile() throws Throwable { + assumeHugeFileExists(); + describe("renaming %s to %s", hugefile, hugefileRenamed); + S3AFileStatus status = fs.getFileStatus(hugefile); + long filesize = status.getLen(); + fs.delete(hugefileRenamed, false); + ContractTestUtils.NanoTimer timer = new ContractTestUtils.NanoTimer(); + fs.rename(hugefile, hugefileRenamed); + long mb = Math.max(filesize / _1MB, 1); + timer.end("Time to rename file of %d MB", mb); + LOG.info("Time per MB to rename = {} nS", toHuman(timer.duration() / mb)); + logFSState(); + S3AFileStatus destFileStatus = fs.getFileStatus(hugefileRenamed); + assertEquals(filesize, destFileStatus.getLen()); + } + + @Test + public void test_999_DeleteHugeFiles() throws IOException { + describe("Deleting %s", hugefile); + fs.delete(hugefile, false); + fs.delete(hugefileRenamed, false); + ContractTestUtils.rm(fs, getTestPath(), true, true); + } + +} diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/S3AScaleTestBase.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/S3AScaleTestBase.java index d861a16230e4f..20a46900779ca 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/S3AScaleTestBase.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/S3AScaleTestBase.java @@ -57,6 +57,10 @@ public static void nameThread() { Thread.currentThread().setName("JUnit"); } + public static final long _1KB = 1024L; + public static final long _1MB = _1KB * _1KB; + public static final long _1GB = _1KB * _1MB; + /** * The number of operations to perform: {@value}. */ From 66ed17f8972f07dbd596092475a7f4efc9fe27d2 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Wed, 31 Aug 2016 12:29:28 +0100 Subject: [PATCH 02/20] HADOOP-13560 tuning test scale and timeouts --- .../site/markdown/tools/hadoop-aws/index.md | 33 ++++++++++++++++++- .../fs/s3a/scale/ITestS3AHugeFileCreate.java | 6 ++++ .../hadoop/fs/s3a/scale/S3AScaleTestBase.java | 11 ++++++- 3 files changed, 48 insertions(+), 2 deletions(-) diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md index ff840dac0f2b3..ce0a19b30cb03 100644 --- a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md +++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md @@ -1731,7 +1731,7 @@ endpoint: ``` -#### Scale test operation count +#### Scale tests Some scale tests perform multiple operations (such as creating many directories). @@ -1768,6 +1768,37 @@ smaller to achieve faster test runs. 10240 +S3A specific scale test properties are + +##### `fs.s3a.scale.test.huge.filesize`: size in MB for "Huge file tests". + +The Huge File tests validate S3A's ability to handle large files —the property +`fs.s3a.scale.test.huge.filesize` declares the file size to use. + +```xml + + fs.s3a.scale.test.huge.filesize + 20 + +``` + +Amazon S3 handles files larger than 5GB differently than smaller ones. +Setting the huge filesize to a number greater than 5120) validates support +for huge files. + +```xml + + fs.s3a.scale.test.huge.filesize + 5130 + +``` + +1. Tests at this scale are slow: they are best executed from hosts running in +the cloud infrastructure where the S3 endpoint is based. +1. The tests are executed in an order to only clean up created files after +the end of all the tests. If the tests are interrupted, the test data will remain. + + ### Testing against non AWS S3 endpoints. diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AHugeFileCreate.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AHugeFileCreate.java index 08f1964eaf912..edf4a5d23558a 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AHugeFileCreate.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AHugeFileCreate.java @@ -28,6 +28,7 @@ import org.junit.Assume; import org.junit.FixMethodOrder; import org.junit.Test; +import org.junit.rules.Timeout; import org.junit.runners.MethodSorters; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -85,6 +86,11 @@ protected Configuration createConfiguration() { return configuration; } + @Override + protected Timeout createTestTimeout() { + return new Timeout(120 * 60 * 1000); + } + @Test public void test_001_CreateHugeFile() throws IOException { long mb = getConf().getLong(KEY_HUGE_FILESIZE, DEFAULT_HUGE_FILESIZE); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/S3AScaleTestBase.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/S3AScaleTestBase.java index 20a46900779ca..a1b09b7b1c2ba 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/S3AScaleTestBase.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/S3AScaleTestBase.java @@ -50,7 +50,8 @@ public class S3AScaleTestBase extends Assert implements S3ATestConstants { public TestName methodName = new TestName(); @Rule - public Timeout testTimeout = new Timeout(30 * 60 * 1000); + public Timeout testTimeout = createTestTimeout(); + @BeforeClass public static void nameThread() { @@ -163,6 +164,14 @@ protected long getOperationCount() { return getConf().getLong(KEY_OPERATION_COUNT, DEFAULT_OPERATION_COUNT); } + /** + * Create the timeout for tests. Some large tests may need a larger value. + * @return the test timeout to use + */ + protected Timeout createTestTimeout() { + return new Timeout(30 * 60 * 1000); + } + /** * Describe a test in the logs * @param text text to print From c35768cfe0c115f0c002999a4f93f5267fd91485 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Wed, 31 Aug 2016 14:21:23 +0100 Subject: [PATCH 03/20] HADOOP-13560 scale tests take maven build arguments --- hadoop-tools/hadoop-aws/pom.xml | 15 ++++ .../hadoop/fs/s3a/S3ATestConstants.java | 11 --- .../fs/s3a/scale/ITestS3AHugeFileCreate.java | 80 +++++++++++++++---- .../hadoop/fs/s3a/scale/S3AScaleTestBase.java | 48 ++++++++++- 4 files changed, 126 insertions(+), 28 deletions(-) diff --git a/hadoop-tools/hadoop-aws/pom.xml b/hadoop-tools/hadoop-aws/pom.xml index 467f2c1632f48..bfeaa935b1834 100644 --- a/hadoop-tools/hadoop-aws/pom.xml +++ b/hadoop-tools/hadoop-aws/pom.xml @@ -35,6 +35,11 @@ UTF-8 true ${project.build.directory}/test + + + 10 + + 1800 @@ -142,6 +147,10 @@ fork-${surefire.forkNumber} + + + ${fs.s3a.scale.test.huge.filesize} + ${fs.s3a.scale.test.timeout} @@ -174,6 +183,12 @@ verify + + + ${fs.s3a.scale.test.huge.filesize} + ${fs.s3a.scale.test.timeout} + + diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestConstants.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestConstants.java index 2be3f429081a9..6d780205fe614 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestConstants.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestConstants.java @@ -74,15 +74,4 @@ public interface S3ATestConstants { */ String KEY_ENCRYPTION_TESTS = TEST_FS_S3A + "encryption.enabled"; - /** - * Name of the property to define the file size for the huge file - * tests: {@value}. Measured in MB. - */ - String KEY_HUGE_FILESIZE = S3A_SCALE_TEST + "huge.filesize"; - - /** - * The default huge size is small —full 5GB+ scale tests are something - * to run in long test runs on EC2 VMs. {@value}. - */ - long DEFAULT_HUGE_FILESIZE = 10L; } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AHugeFileCreate.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AHugeFileCreate.java index edf4a5d23558a..742417661be93 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AHugeFileCreate.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AHugeFileCreate.java @@ -22,13 +22,16 @@ import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.StorageStatistics; import org.apache.hadoop.fs.contract.ContractTestUtils; import org.apache.hadoop.fs.s3a.Constants; import org.apache.hadoop.fs.s3a.S3AFileStatus; +import org.apache.hadoop.fs.s3a.Statistic; +import org.apache.hadoop.util.Progressable; import org.junit.Assume; import org.junit.FixMethodOrder; import org.junit.Test; -import org.junit.rules.Timeout; +import org.junit.internal.AssumptionViolatedException; import org.junit.runners.MethodSorters; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -73,35 +76,35 @@ public void setUp() throws Exception { @Override public void tearDown() throws Exception { - // do nothing + // do nothing. Specifically: do not delete the test dir } @Override protected Configuration createConfiguration() { Configuration configuration = super.createConfiguration(); configuration.setBoolean(Constants.FAST_UPLOAD, true); - configuration.setLong(MIN_MULTIPART_THRESHOLD, 10 * _1MB); + configuration.setLong(MIN_MULTIPART_THRESHOLD, 1 * _1MB); configuration.setLong(SOCKET_SEND_BUFFER, BLOCKSIZE); configuration.setLong(SOCKET_RECV_BUFFER, BLOCKSIZE); return configuration; } - @Override - protected Timeout createTestTimeout() { - return new Timeout(120 * 60 * 1000); - } - @Test public void test_001_CreateHugeFile() throws IOException { - long mb = getConf().getLong(KEY_HUGE_FILESIZE, DEFAULT_HUGE_FILESIZE); + long mb = getTestProperty(KEY_HUGE_FILESIZE, DEFAULT_HUGE_FILESIZE); long filesize = _1MB * mb; describe("Creating file %s of size %d MB", hugefile, mb); try { - long actualSize = fs.getFileStatus(hugefile).getLen(); - Assume.assumeTrue("File of desired size already exists; skipping", - actualSize != filesize); + S3AFileStatus status = fs.getFileStatus(hugefile); + long actualSize = status.getLen(); + if (actualSize == filesize) { + String message = "File of size " + mb + " MB exists: " + status; + LOG.info(message); + throw new AssumptionViolatedException(message); + } } catch (FileNotFoundException e) { + // the file doesn't exist and so must be created. } byte[] data = new byte[BLOCKSIZE]; for (int i = 0; i < BLOCKSIZE; i++) { @@ -115,15 +118,46 @@ public void test_001_CreateHugeFile() throws IOException { long blocks = filesize / BLOCKSIZE; long blocksPerMB = _1MB / BLOCKSIZE; + // perform the upload. + // there's lots of logging here, so that a tail -f on the output log + // can give a view of what is happening. + StorageStatistics storageStatistics = fs.getStorageStatistics(); + String putRequests = Statistic.OBJECT_PUT_REQUESTS.getSymbol(); + String putBytes = Statistic.OBJECT_PUT_BYTES.getSymbol(); + ContractTestUtils.NanoTimer timer = new ContractTestUtils.NanoTimer(); - try(FSDataOutputStream out = fs.create(hugefile, true)) { + + try(FSDataOutputStream out = fs.create(hugefile, + true, + BLOCKSIZE, + new ProgressCallback())) { + + for (long block = 0; block < blocks; block++) { out.write(data); if (block > 0 && blocksPerMB % block == 0) { - LOG.info("."); + long written = block * BLOCKSIZE; + long percentage = written * 100 / filesize; + LOG.info(String.format("[%02d%%] Written %d MB out of %d MB;" + + " PUT = %d bytes in %d operations", + percentage, written, filesize, + storageStatistics.getLong(putBytes), + storageStatistics.getLong( + putRequests) + )); } } + // now close the file + ContractTestUtils.NanoTimer flushTimer + = new ContractTestUtils.NanoTimer(); +// out.flush(); + flushTimer.end("Time to flush() output stream"); + ContractTestUtils.NanoTimer closeTimer + = new ContractTestUtils.NanoTimer(); + out.close(); + closeTimer.end("Time to close() output stream"); } + timer.end("Time to write %d MB in blocks of %d", mb, BLOCKSIZE); LOG.info("Time per MB to write = {} nS", toHuman(timer.duration() / mb)); @@ -132,6 +166,22 @@ public void test_001_CreateHugeFile() throws IOException { assertEquals("File size in " + status, filesize, status.getLen()); } + /** + * Progress callback from AWS. Likely to come in on a different thread. + */ + private static class ProgressCallback implements Progressable { + private int counter = 0; + + @Override + public void progress() { + counter ++; + } + + public int getCounter() { + return counter; + } + } + void assumeHugeFileExists() throws IOException { Assume.assumeTrue("No file " + hugefile, fs.exists(hugefile)); } @@ -159,7 +209,7 @@ public void test_050_readHugeFile() throws Throwable { } private void logFSState() { - LOG.info("File System state after operation; {}", fs); + LOG.info("File System state after operation:\n{}", fs); } @Test diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/S3AScaleTestBase.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/S3AScaleTestBase.java index a1b09b7b1c2ba..808bb8c49b2ae 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/S3AScaleTestBase.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/S3AScaleTestBase.java @@ -107,6 +107,24 @@ public static void nameThread() { public static final String DEFAULT_CSVTEST_ENDPOINT = "s3.amazonaws.com"; + /** + * Name of the property to define the timeout for scale tests: {@value}. + * Measured in seconds. + */ + public static final String KEY_TEST_TIMEOUT = S3A_SCALE_TEST + "timeout"; + + /** + * Name of the property to define the file size for the huge file + * tests: {@value}. Measured in MB. + */ + public static final String KEY_HUGE_FILESIZE = S3A_SCALE_TEST + "huge.filesize"; + + /** + * The default huge size is small —full 5GB+ scale tests are something + * to run in long test runs on EC2 VMs. {@value}. + */ + public static final long DEFAULT_HUGE_FILESIZE = 10L; + /** * The default number of operations to perform: {@value}. */ @@ -118,6 +136,11 @@ public static void nameThread() { */ public static final int DEFAULT_DIRECTORY_COUNT = 2; + /** + * Default scale test timeout in seconds: {@value}. + */ + public static final long DEFAULT_TEST_TIMEOUT = 30 * 60; + protected S3AFileSystem fs; protected static final Logger LOG = @@ -144,11 +167,17 @@ public Configuration getConf() { @Before public void setUp() throws Exception { - conf = createConfiguration(); + demandCreateConfiguration(); LOG.debug("Scale test operation count = {}", getOperationCount()); fs = S3ATestUtils.createTestFileSystem(conf); } + private void demandCreateConfiguration() { + if (conf == null) { + conf = createConfiguration(); + } + } + @After public void tearDown() throws Exception { ContractTestUtils.rm(fs, getTestPath(), true, true); @@ -169,7 +198,22 @@ protected long getOperationCount() { * @return the test timeout to use */ protected Timeout createTestTimeout() { - return new Timeout(30 * 60 * 1000); + return new Timeout((int)getTestProperty(KEY_TEST_TIMEOUT, + DEFAULT_TEST_TIMEOUT) * 1000); + } + + /** + * Get a test property which can defined first as a system property + * and then potentially overridden in a configuration property. + * @param key key to look up + * @param defVal default value + * @return the evaluated test property. + */ + protected long getTestProperty(String key, long defVal) { + demandCreateConfiguration(); + String propval = System.getProperty(key, Long.toString(defVal)); + long longVal = propval!=null? Long.valueOf(propval): defVal; + return getConf().getLong(key, longVal); } /** From 2fae934da9b69e527e6ef2f12e2e5939a006051b Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Wed, 31 Aug 2016 14:33:47 +0100 Subject: [PATCH 04/20] HADOOP-13567 S3AFileSystem to override getStoragetStatistics() and so serve up its statistics --- .../java/org/apache/hadoop/fs/s3a/S3AFileSystem.java | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java index f238362eb35bf..7d08e8095b632 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java @@ -726,6 +726,15 @@ protected void incrementStatistic(Statistic statistic, long count) { storageStatistics.incrementCounter(statistic, count); } + /** + * Get the storage statistics of this filesystem. + * @return the storage statistics + */ + @Override + public S3AStorageStatistics getStorageStatistics() { + return storageStatistics; + } + /** * Request object metadata; increments counters in the process. * @param key key From 5a8fd6935dbed47957339ec722d39f735fb9d04f Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Wed, 31 Aug 2016 14:34:26 +0100 Subject: [PATCH 05/20] HADOOP-13566 NPE in S3AFastOutputStream.write --- .../apache/hadoop/fs/s3a/S3AFastOutputStream.java | 12 ++++++++++++ .../org/apache/hadoop/fs/s3a/S3AOutputStream.java | 13 +++++++++++++ 2 files changed, 25 insertions(+) diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFastOutputStream.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFastOutputStream.java index c25d0fbf90172..6412158639859 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFastOutputStream.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFastOutputStream.java @@ -153,6 +153,16 @@ public S3AFastOutputStream(AmazonS3 client, bucket, key); } + /** + * Check for the filesystem being open. + * @throws IOException if the filesystem is closed. + */ + void checkOpen() throws IOException { + if (closed) { + throw new IOException("Filesystem closed"); + } + } + /** * Writes a byte to the memory buffer. If this causes the buffer to reach * its limit, the actual upload is submitted to the threadpool. @@ -161,6 +171,7 @@ public S3AFastOutputStream(AmazonS3 client, */ @Override public synchronized void write(int b) throws IOException { + checkOpen(); buffer.write(b); if (buffer.size() == bufferLimit) { uploadBuffer(); @@ -180,6 +191,7 @@ public synchronized void write(int b) throws IOException { @Override public synchronized void write(byte[] b, int off, int len) throws IOException { + checkOpen(); if (b == null) { throw new NullPointerException(); } else if ((off < 0) || (off > b.length) || (len < 0) || diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AOutputStream.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AOutputStream.java index 23ba6828e777f..bd9243c2bc8d6 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AOutputStream.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AOutputStream.java @@ -84,8 +84,19 @@ public S3AOutputStream(Configuration conf, new FileOutputStream(backupFile)); } + /** + * Check for the filesystem being open. + * @throws IOException if the filesystem is closed. + */ + void checkOpen() throws IOException { + if (closed) { + throw new IOException("Filesystem closed"); + } + } + @Override public void flush() throws IOException { + checkOpen(); backupStream.flush(); } @@ -133,11 +144,13 @@ public synchronized void close() throws IOException { @Override public void write(int b) throws IOException { + checkOpen(); backupStream.write(b); } @Override public void write(byte[] b, int off, int len) throws IOException { + checkOpen(); backupStream.write(b, off, len); } From 28397a0ad768c8c6a00e607d3635fa791ec06857 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Wed, 31 Aug 2016 15:31:39 +0100 Subject: [PATCH 06/20] HADOOP-13560 use STest as prefix for scale tests --- hadoop-tools/hadoop-aws/pom.xml | 2 ++ .../apache/hadoop/fs/s3a/S3ATestUtils.java | 19 ++++++++++- .../hadoop/fs/s3a/scale/S3AScaleTestBase.java | 10 +++++- .../fs/s3a/scale/STestS3AHugeDistCP.java | 32 ++++++++++++++++++ ...reate.java => STestS3AHugeFileCreate.java} | 33 ++++++++++++------- 5 files changed, 82 insertions(+), 14 deletions(-) create mode 100644 hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/STestS3AHugeDistCP.java rename hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/{ITestS3AHugeFileCreate.java => STestS3AHugeFileCreate.java} (87%) diff --git a/hadoop-tools/hadoop-aws/pom.xml b/hadoop-tools/hadoop-aws/pom.xml index bfeaa935b1834..0bdeec5d0e5c5 100644 --- a/hadoop-tools/hadoop-aws/pom.xml +++ b/hadoop-tools/hadoop-aws/pom.xml @@ -173,6 +173,7 @@ **/ITestS3AFileSystemContract.java **/ITestS3AMiniYarnCluster.java **/ITest*Root*.java + **/ITestS3AHugeFileCreate.java @@ -198,6 +199,7 @@ **/ITestS3AFileSystemContract.java **/ITestS3AMiniYarnCluster.java **/ITest*Root*.java + **/STest*.java diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java index e45db483e7290..5b4fe9e5d6883 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java @@ -53,8 +53,25 @@ public class S3ATestUtils { */ public static S3AFileSystem createTestFileSystem(Configuration conf) throws IOException { - String fsname = conf.getTrimmed(TEST_FS_S3A_NAME, ""); + return createTestFileSystem(conf, true); + } + + /** + * Create the test filesystem with or without multipart purging + * + * If the test.fs.s3a.name property is not set, this will + * trigger a JUnit failure. + * @param conf configuration + * @param purge flag to enable Multipart purging + * @return the FS + * @throws IOException IO Problems + * @throws AssumptionViolatedException if the FS is not named + */ + public static S3AFileSystem createTestFileSystem(Configuration conf, + boolean purge) + throws IOException { + String fsname = conf.getTrimmed(TEST_FS_S3A_NAME, ""); boolean liveTest = !StringUtils.isEmpty(fsname); URI testURI = null; diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/S3AScaleTestBase.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/S3AScaleTestBase.java index 808bb8c49b2ae..627c03a189a6d 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/S3AScaleTestBase.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/S3AScaleTestBase.java @@ -119,6 +119,13 @@ public static void nameThread() { */ public static final String KEY_HUGE_FILESIZE = S3A_SCALE_TEST + "huge.filesize"; + /** + * Name of the property to define the partition size for the huge file + * tests: {@value}. Measured in MB. + */ + public static final String KEY_HUGE_PARTITION_SIZE = + S3A_SCALE_TEST + "huge.partition.size"; + /** * The default huge size is small —full 5GB+ scale tests are something * to run in long test runs on EC2 VMs. {@value}. @@ -169,7 +176,8 @@ public Configuration getConf() { public void setUp() throws Exception { demandCreateConfiguration(); LOG.debug("Scale test operation count = {}", getOperationCount()); - fs = S3ATestUtils.createTestFileSystem(conf); + // multipart purges are disabled on the scale tests + fs = S3ATestUtils.createTestFileSystem(conf, false); } private void demandCreateConfiguration() { diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/STestS3AHugeDistCP.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/STestS3AHugeDistCP.java new file mode 100644 index 0000000000000..d6598d6d9952e --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/STestS3AHugeDistCP.java @@ -0,0 +1,32 @@ +/* + * 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.scale; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Test very large scale distCP upload using the huge file options + */ +public class STestS3AHugeDistCP extends S3AScaleTestBase { + private static final Logger LOG = LoggerFactory.getLogger( + STestS3AHugeDistCP.class); + + +} diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AHugeFileCreate.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/STestS3AHugeFileCreate.java similarity index 87% rename from hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AHugeFileCreate.java rename to hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/STestS3AHugeFileCreate.java index 742417661be93..6570c7473f26d 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AHugeFileCreate.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/STestS3AHugeFileCreate.java @@ -56,9 +56,9 @@ * in case an individual test is executed. */ @FixMethodOrder(MethodSorters.NAME_ASCENDING) -public class ITestS3AHugeFileCreate extends S3AScaleTestBase { +public class STestS3AHugeFileCreate extends S3AScaleTestBase { private static final Logger LOG = LoggerFactory.getLogger( - ITestS3AHugeFileCreate.class); + STestS3AHugeFileCreate.class); private Path scaleTestDir; private Path hugefile; private Path hugefileRenamed; @@ -83,14 +83,14 @@ public void tearDown() throws Exception { protected Configuration createConfiguration() { Configuration configuration = super.createConfiguration(); configuration.setBoolean(Constants.FAST_UPLOAD, true); - configuration.setLong(MIN_MULTIPART_THRESHOLD, 1 * _1MB); + configuration.setLong(MIN_MULTIPART_THRESHOLD, 5 * _1MB); configuration.setLong(SOCKET_SEND_BUFFER, BLOCKSIZE); configuration.setLong(SOCKET_RECV_BUFFER, BLOCKSIZE); return configuration; } @Test - public void test_001_CreateHugeFile() throws IOException { + public void test_010_CreateHugeFile() throws IOException { long mb = getTestProperty(KEY_HUGE_FILESIZE, DEFAULT_HUGE_FILESIZE); long filesize = _1MB * mb; @@ -133,14 +133,16 @@ public void test_001_CreateHugeFile() throws IOException { new ProgressCallback())) { - for (long block = 0; block < blocks; block++) { + for (long block = 1; block <= blocks; block++) { out.write(data); - if (block > 0 && blocksPerMB % block == 0) { + if (block % blocksPerMB == 0) { long written = block * BLOCKSIZE; long percentage = written * 100 / filesize; - LOG.info(String.format("[%02d%%] Written %d MB out of %d MB;" + + LOG.info(String.format("[%03d%%] Written %.2f MB out of %.2f MB;" + " PUT = %d bytes in %d operations", - percentage, written, filesize, + percentage, + 1.0 * written / _1MB, + 1.0 * filesize / _1MB, storageStatistics.getLong(putBytes), storageStatistics.getLong( putRequests) @@ -148,10 +150,6 @@ public void test_001_CreateHugeFile() throws IOException { } } // now close the file - ContractTestUtils.NanoTimer flushTimer - = new ContractTestUtils.NanoTimer(); -// out.flush(); - flushTimer.end("Time to flush() output stream"); ContractTestUtils.NanoTimer closeTimer = new ContractTestUtils.NanoTimer(); out.close(); @@ -161,7 +159,11 @@ public void test_001_CreateHugeFile() throws IOException { timer.end("Time to write %d MB in blocks of %d", mb, BLOCKSIZE); LOG.info("Time per MB to write = {} nS", toHuman(timer.duration() / mb)); + LOG.info("Effective Bandwidth: {} MB/s", timer.bandwidth(filesize)); logFSState(); + Long putRequestCount = storageStatistics.getLong(putRequests); + LOG.info("Time per PUT {}", + toHuman(timer.nanosPerOperation(putRequestCount))); S3AFileStatus status = fs.getFileStatus(hugefile); assertEquals("File size in " + status, filesize, status.getLen()); } @@ -205,6 +207,7 @@ public void test_050_readHugeFile() throws Throwable { long mb = Math.max(filesize / _1MB, 1); timer.end("Time to read file of %d MB ", mb); LOG.info("Time per MB to read = {} nS", toHuman(timer.duration() / mb)); + LOG.info("Effective Bandwidth: {} MB/s", timer.bandwidth(filesize)); logFSState(); } @@ -224,6 +227,7 @@ public void test_100_renameHugeFile() throws Throwable { long mb = Math.max(filesize / _1MB, 1); timer.end("Time to rename file of %d MB", mb); LOG.info("Time per MB to rename = {} nS", toHuman(timer.duration() / mb)); + LOG.info("Effective Bandwidth: {} MB/s" , timer.bandwidth(filesize)); logFSState(); S3AFileStatus destFileStatus = fs.getFileStatus(hugefileRenamed); assertEquals(filesize, destFileStatus.getLen()); @@ -232,8 +236,13 @@ public void test_100_renameHugeFile() throws Throwable { @Test public void test_999_DeleteHugeFiles() throws IOException { describe("Deleting %s", hugefile); + ContractTestUtils.NanoTimer timer = new ContractTestUtils.NanoTimer(); fs.delete(hugefile, false); + timer.end("Time to delete %s", hugefile); + ContractTestUtils.NanoTimer timer2 = new ContractTestUtils.NanoTimer(); + fs.delete(hugefileRenamed, false); + timer2.end("Time to delete %s", hugefileRenamed); ContractTestUtils.rm(fs, getTestPath(), true, true); } From e1c8f6168c179209cc8ce8f172e7104925644fbf Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Wed, 31 Aug 2016 18:06:51 +0100 Subject: [PATCH 07/20] HADOOP-13560 test improvements --- hadoop-tools/hadoop-aws/pom.xml | 3 +- .../hadoop/fs/s3a/scale/S3AScaleTestBase.java | 5 ++- .../fs/s3a/scale/STestS3AHugeFileCreate.java | 36 +++++++++++++++---- 3 files changed, 34 insertions(+), 10 deletions(-) diff --git a/hadoop-tools/hadoop-aws/pom.xml b/hadoop-tools/hadoop-aws/pom.xml index 0bdeec5d0e5c5..70860a2f63649 100644 --- a/hadoop-tools/hadoop-aws/pom.xml +++ b/hadoop-tools/hadoop-aws/pom.xml @@ -173,7 +173,7 @@ **/ITestS3AFileSystemContract.java **/ITestS3AMiniYarnCluster.java **/ITest*Root*.java - **/ITestS3AHugeFileCreate.java + **/ITestS3AFileContextStatistics.java @@ -199,6 +199,7 @@ **/ITestS3AFileSystemContract.java **/ITestS3AMiniYarnCluster.java **/ITest*Root*.java + **/ITestS3AFileContextStatistics.java **/STest*.java diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/S3AScaleTestBase.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/S3AScaleTestBase.java index 627c03a189a6d..b1ab050b9d63b 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/S3AScaleTestBase.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/S3AScaleTestBase.java @@ -52,9 +52,8 @@ public class S3AScaleTestBase extends Assert implements S3ATestConstants { @Rule public Timeout testTimeout = createTestTimeout(); - - @BeforeClass - public static void nameThread() { + @Before + public void nameThread() { Thread.currentThread().setName("JUnit"); } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/STestS3AHugeFileCreate.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/STestS3AHugeFileCreate.java index 6570c7473f26d..3bd9803da1289 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/STestS3AHugeFileCreate.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/STestS3AHugeFileCreate.java @@ -18,6 +18,8 @@ package org.apache.hadoop.fs.s3a.scale; +import com.amazonaws.event.ProgressEvent; +import com.amazonaws.event.ProgressListener; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSDataOutputStream; @@ -132,7 +134,6 @@ public void test_010_CreateHugeFile() throws IOException { BLOCKSIZE, new ProgressCallback())) { - for (long block = 1; block <= blocks; block++) { out.write(data); if (block % blocksPerMB == 0) { @@ -150,6 +151,7 @@ public void test_010_CreateHugeFile() throws IOException { } } // now close the file + LOG.info("Closing file and completing write operation"); ContractTestUtils.NanoTimer closeTimer = new ContractTestUtils.NanoTimer(); out.close(); @@ -158,11 +160,17 @@ public void test_010_CreateHugeFile() throws IOException { timer.end("Time to write %d MB in blocks of %d", mb, BLOCKSIZE); - LOG.info("Time per MB to write = {} nS", toHuman(timer.duration() / mb)); - LOG.info("Effective Bandwidth: {} MB/s", timer.bandwidth(filesize)); logFSState(); + if (mb > 0) { + LOG.info("Time per MB to write = {} nS", toHuman(timer.duration() / mb)); + } + LOG.info("Effective Bandwidth: {} MB/s", timer.bandwidth(filesize)); Long putRequestCount = storageStatistics.getLong(putRequests); - LOG.info("Time per PUT {}", + Long putByteCount = storageStatistics.getLong(putBytes); + LOG.info("PUT {} bytes in {} operations; {} MB/operation", + putByteCount, putRequestCount, + putByteCount / (putRequestCount * _1MB)); + LOG.info("Time per PUT {} nS", toHuman(timer.nanosPerOperation(putRequestCount))); S3AFileStatus status = fs.getFileStatus(hugefile); assertEquals("File size in " + status, filesize, status.getLen()); @@ -171,17 +179,28 @@ public void test_010_CreateHugeFile() throws IOException { /** * Progress callback from AWS. Likely to come in on a different thread. */ - private static class ProgressCallback implements Progressable { + private static class ProgressCallback implements Progressable, + ProgressListener { private int counter = 0; @Override public void progress() { - counter ++; + counter++; } public int getCounter() { return counter; } + + @Override + public void progressChanged(ProgressEvent progressEvent) { + counter++; + if (progressEvent.getEventType().isByteCountEvent()) { + LOG.debug("Event {}", progressEvent); + } else { + LOG.info("Event {}", progressEvent); + } + } } void assumeHugeFileExists() throws IOException { @@ -231,6 +250,11 @@ public void test_100_renameHugeFile() throws Throwable { logFSState(); S3AFileStatus destFileStatus = fs.getFileStatus(hugefileRenamed); assertEquals(filesize, destFileStatus.getLen()); + + // rename back + ContractTestUtils.NanoTimer timer2 = new ContractTestUtils.NanoTimer(); + fs.rename(hugefileRenamed, hugefile); + timer2.end("Renaming back"); } @Test From 33afa2969210aa976d79aef9677af7f0b47630ec Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Wed, 31 Aug 2016 18:07:29 +0100 Subject: [PATCH 08/20] HADOOP-13560 fix typo in the name of a statistic --- .../src/main/java/org/apache/hadoop/fs/s3a/Statistic.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Statistic.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Statistic.java index cbc34d665dc25..28b61b82029e1 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Statistic.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Statistic.java @@ -80,7 +80,7 @@ public enum Statistic { OBJECT_PUT_BYTES("object_put_bytes", "number of bytes uploaded"), STREAM_ABORTED("stream_aborted", "Count of times the TCP stream was aborted"), - STREAM_BACKWARD_SEEK_OPERATIONS("stream_backward_seek_pperations", + STREAM_BACKWARD_SEEK_OPERATIONS("stream_backward_seek_operations", "Number of executed seek operations which went backwards in a stream"), STREAM_CLOSED("streamClosed", "Count of times the TCP stream was closed"), STREAM_CLOSE_OPERATIONS("stream_close_operations", From d17da15d094c038e1f5d2fc3ce69ad0fac4aec2a Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Wed, 31 Aug 2016 18:08:42 +0100 Subject: [PATCH 09/20] HADOOP-13569 S3AFastOutputStream to take ProgressListener in file create() --- .../org/apache/hadoop/fs/s3a/S3AFastOutputStream.java | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFastOutputStream.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFastOutputStream.java index 6412158639859..7fdbd43dfc7ce 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFastOutputStream.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFastOutputStream.java @@ -94,7 +94,9 @@ public class S3AFastOutputStream extends OutputStream { * @param fs S3AFilesystem * @param bucket S3 bucket name * @param key S3 key name - * @param progress report progress in order to prevent timeouts + * @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 cannedACL used CannedAccessControlList * @param partSize size of a single part in a multi-part upload (except * last part) @@ -148,7 +150,9 @@ public S3AFastOutputStream(AmazonS3 client, this.buffer = new ByteArrayOutputStream(initialBufferSize); this.executorService = MoreExecutors.listeningDecorator(threadPoolExecutor); this.multiPartUpload = null; - this.progressListener = new ProgressableListener(progress); + this.progressListener = (progress instanceof ProgressListener) ? + (ProgressListener) progress + : new ProgressableListener(progress); LOG.debug("Initialized S3AFastOutputStream for bucket '{}' key '{}'", bucket, key); } From ace9f5f8b7cf8a4f46b89dda41f4aea873c1819c Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Thu, 1 Sep 2016 18:05:01 +0100 Subject: [PATCH 10/20] 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 --- .../hadoop/fs/s3a/S3AFastOutputStream.java | 101 ++++++++++++++---- .../apache/hadoop/fs/s3a/S3AFileSystem.java | 90 +++++++++++++++- .../hadoop/fs/s3a/S3AInstrumentation.java | 64 ++++++++++- .../apache/hadoop/fs/s3a/S3AOutputStream.java | 8 -- .../org/apache/hadoop/fs/s3a/S3AUtils.java | 12 +++ .../org/apache/hadoop/fs/s3a/Statistic.java | 6 ++ .../site/markdown/tools/hadoop-aws/index.md | 61 +++++++++++ .../fs/s3a/ITestS3AFastOutputStream.java | 1 - .../ITestS3AFileContextStatistics.java | 1 + .../hadoop/fs/s3a/scale/S3AScaleTestBase.java | 16 +++ .../fs/s3a/scale/STestS3AHugeFileCreate.java | 32 +++--- 11 files changed, 343 insertions(+), 49 deletions(-) diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFastOutputStream.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFastOutputStream.java index 7fdbd43dfc7ce..7b4c9149420b0 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFastOutputStream.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFastOutputStream.java @@ -19,12 +19,14 @@ package org.apache.hadoop.fs.s3a; import com.amazonaws.AmazonClientException; +import com.amazonaws.AmazonServiceException; import com.amazonaws.event.ProgressEvent; import com.amazonaws.event.ProgressListener; import com.amazonaws.services.s3.AmazonS3; import com.amazonaws.services.s3.model.AbortMultipartUploadRequest; import com.amazonaws.services.s3.model.CannedAccessControlList; import com.amazonaws.services.s3.model.CompleteMultipartUploadRequest; +import com.amazonaws.services.s3.model.CompleteMultipartUploadResult; import com.amazonaws.services.s3.model.InitiateMultipartUploadRequest; import com.amazonaws.services.s3.model.ObjectMetadata; import com.amazonaws.services.s3.model.PartETag; @@ -37,6 +39,8 @@ import com.google.common.util.concurrent.MoreExecutors; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.io.retry.RetryPolicies; +import org.apache.hadoop.io.retry.RetryPolicy; import org.apache.hadoop.util.Progressable; import org.slf4j.Logger; @@ -50,6 +54,7 @@ import java.util.concurrent.Callable; import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; +import java.util.concurrent.TimeUnit; import static org.apache.hadoop.fs.s3a.S3AUtils.*; import static org.apache.hadoop.fs.s3a.Statistic.*; @@ -82,6 +87,7 @@ public class S3AFastOutputStream extends OutputStream { private boolean closed; private ByteArrayOutputStream buffer; private int bufferLimit; + private final RetryPolicy retryPolicy; /** @@ -153,6 +159,8 @@ public S3AFastOutputStream(AmazonS3 client, this.progressListener = (progress instanceof ProgressListener) ? (ProgressListener) progress : new ProgressableListener(progress); + this.retryPolicy = RetryPolicies.retryUpToMaximumCountWithProportionalSleep( + 5, 2000, TimeUnit.MILLISECONDS); LOG.debug("Initialized S3AFastOutputStream for bucket '{}' key '{}'", bucket, key); } @@ -261,7 +269,6 @@ public synchronized void close() throws IOException { } else { int size = buffer.size(); if (size > 0) { - fs.incrementPutStartStatistics(size); //send last part multiPartUpload.uploadPartAsync(new ByteArrayInputStream(buffer .toByteArray()), size); @@ -316,8 +323,7 @@ private void putObject() throws IOException { executorService.submit(new Callable() { @Override public PutObjectResult call() throws Exception { - fs.incrementPutStartStatistics(size); - return client.putObject(putObjectRequest); + return fs.putObjectDirect(putObjectRequest); } }); //wait for completion @@ -383,31 +389,84 @@ private List waitForAllPartUploads() throws IOException { } } - private void complete(List partETags) throws IOException { - try { - LOG.debug("Completing multi-part upload for key '{}', id '{}'", - key, uploadId); - client.completeMultipartUpload( - new CompleteMultipartUploadRequest(bucket, - key, - uploadId, - partETags)); - } catch (AmazonClientException e) { - throw translateException("Completing multi-part upload", key, e); - } + /** + * This completes the upload. + * Sometimes it fails; here retries are handled to avoid losing all data + * on a transient failure. + * @param partETags list of partial uploads + * @throws IOException on any problem + */ + private CompleteMultipartUploadResult complete(List partETags) + throws IOException { + int retryCount = 0; + AmazonClientException lastException; + do { + try { + LOG.debug("Completing multi-part upload for key '{}', id '{}'", + key, uploadId); + return client.completeMultipartUpload( + new CompleteMultipartUploadRequest(bucket, + key, + uploadId, + partETags)); + } catch (AmazonClientException e) { + lastException = e; + } + } while (shouldRetry(lastException, retryCount++)); + // this point is only reached if the operation failed more than + // the allowed retry count + throw translateException("Completing multi-part upload", key, + lastException); } + /** + * Abort a multi-part upload. Retries are attempted on failures. + */ public void abort() { + int retryCount = 0; + AmazonClientException lastException; LOG.warn("Aborting multi-part upload with id '{}'", uploadId); + fs.incrementStatistic(OBJECT_MULTIPART_UPLOAD_ABORTED); + do { + try { + LOG.debug("Completing multi-part upload for key '{}', id '{}'", + key, uploadId); + client.abortMultipartUpload(new AbortMultipartUploadRequest(bucket, + key, uploadId)); + return; + } catch (AmazonClientException e) { + lastException = e; + } + } while (shouldRetry(lastException, retryCount++)); + // this point is only reached if the operation failed more than + // the allowed retry count + LOG.warn("Unable to abort multipart upload, you may need to purge " + + "uploaded parts: {}", lastException, lastException); + } + + /** + * Predicate to determine whether a failed operation should + * be attempted again. + * If a retry is advised, the exception is automatically logged and + * the filesystem statistic {@link Statistic.IGNORED_ERRORS} incremented. + * @param e exception raised. + * @param retryCount number of retries already attempted + * @return true if another attempt should be made + */ + private boolean shouldRetry(AmazonClientException e, int retryCount) { try { - fs.incrementStatistic(OBJECT_MULTIPART_UPLOAD_ABORTED); - client.abortMultipartUpload(new AbortMultipartUploadRequest(bucket, - key, uploadId)); - } catch (Exception e2) { - LOG.warn("Unable to abort multipart upload, you may need to purge " + - "uploaded parts: {}", e2, e2); + boolean retry = retryPolicy.shouldRetry(e, retryCount, 0, true) + == RetryPolicy.RetryAction.RETRY; + if (retry) { + fs.incrementStatistic(IGNORED_ERRORS); + LOG.info("Retrying operation after exception " + e, e); + } + return retry; + } catch (Exception ignored) { + return false; } } + } private static class ProgressableListener implements ProgressListener { diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java index 7d08e8095b632..7043131277054 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java @@ -44,6 +44,7 @@ import com.amazonaws.services.s3.model.ObjectMetadata; import com.amazonaws.services.s3.model.PutObjectRequest; import com.amazonaws.services.s3.model.CopyObjectRequest; +import com.amazonaws.services.s3.model.PutObjectResult; import com.amazonaws.services.s3.model.S3ObjectSummary; import com.amazonaws.services.s3.model.UploadPartRequest; import com.amazonaws.services.s3.model.UploadPartResult; @@ -127,6 +128,7 @@ public class S3AFileSystem extends FileSystem { // 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 fastUploadEnabled; /** Called after a new FileSystem instance is constructed. * @param name a uri whose authority section names the host, port, etc. @@ -214,6 +216,8 @@ public StorageStatistics provide() { conf.getTrimmed(SERVER_SIDE_ENCRYPTION_ALGORITHM); inputPolicy = S3AInputPolicy.getPolicy( conf.getTrimmed(INPUT_FADVISE, INPUT_FADV_NORMAL)); + fastUploadEnabled = getConf().getBoolean(FAST_UPLOAD, + DEFAULT_FAST_UPLOAD); } catch (AmazonClientException e) { throw translateException("initializing ", new Path(name), e); } @@ -487,7 +491,7 @@ public FSDataOutputStream create(Path f, FsPermission permission, } instrumentation.fileCreated(); - if (getConf().getBoolean(FAST_UPLOAD, DEFAULT_FAST_UPLOAD)) { + if (fastUploadEnabled) { return new FSDataOutputStream( new S3AFastOutputStream(s3, this, @@ -726,6 +730,24 @@ protected void incrementStatistic(Statistic statistic, long count) { storageStatistics.incrementCounter(statistic, count); } + /** + * Decrement a gauge by a specific value. + * @param statistic The operation to decrement + * @param count the count to decrement + */ + protected void decrementGauge(Statistic statistic, long count) { + instrumentation.decrementGauge(statistic, count); + } + + /** + * Increment a gauge by a specific value. + * @param statistic The operation to increment + * @param count the count to increment + */ + protected void incrementGauge(Statistic statistic, long count) { + instrumentation.incrementGauge(statistic, count); + } + /** * Get the storage statistics of this filesystem. * @return the storage statistics @@ -890,7 +912,39 @@ public Upload putObject(PutObjectRequest putObjectRequest) { len = putObjectRequest.getMetadata().getContentLength(); } incrementPutStartStatistics(len); - return transfers.upload(putObjectRequest); + try { + Upload upload = transfers.upload(putObjectRequest); + incrementPutCompletedStatistics(true, len); + return upload; + } catch (AmazonClientException e) { + incrementPutCompletedStatistics(false, len); + throw e; + } + } + + /** + * PUT an object directly (i.e. not via the transfer manager). + * Byte length is calculated from the file length, or, if there is no + * file, from the content length of the header. + * @param putObjectRequest the request + * @return the upload initiated + */ + public PutObjectResult putObjectDirect(PutObjectRequest putObjectRequest) { + long len; + if (putObjectRequest.getFile() != null) { + len = putObjectRequest.getFile().length(); + } else { + len = putObjectRequest.getMetadata().getContentLength(); + } + incrementPutStartStatistics(len); + try { + PutObjectResult result = s3.putObject(putObjectRequest); + incrementPutCompletedStatistics(true, len); + return result; + } catch (AmazonClientException e) { + incrementPutCompletedStatistics(false, len); + throw e; + } } /** @@ -900,8 +954,16 @@ public Upload putObject(PutObjectRequest putObjectRequest) { * @return the result of the operation. */ public UploadPartResult uploadPart(UploadPartRequest request) { - incrementPutStartStatistics(request.getPartSize()); - return s3.uploadPart(request); + long len = request.getPartSize(); + incrementPutStartStatistics(len); + try { + UploadPartResult uploadPartResult = s3.uploadPart(request); + incrementPutCompletedStatistics(true, len); + return uploadPartResult; + } catch (AmazonClientException e) { + incrementPutCompletedStatistics(false, len); + throw e; + } } /** @@ -914,9 +976,28 @@ public void incrementPutStartStatistics(long bytes) { LOG.debug("PUT start {} bytes", bytes); incrementWriteOperations(); incrementStatistic(OBJECT_PUT_REQUESTS); + incrementGauge(OBJECT_PUT_REQUESTS_ACTIVE, 1); + if (bytes > 0) { + incrementGauge(OBJECT_PUT_BYTES_PENDING, bytes); + } + } + + /** + * At the end of a put/multipart upload operation, update the + * relevant counters and gauges. + * + * @param success did the operation succeed? + * @param bytes bytes in the request. + */ + public void incrementPutCompletedStatistics(boolean success, long bytes) { + LOG.debug("PUT completed success={}; {} bytes", success, bytes); + incrementWriteOperations(); if (bytes > 0) { incrementStatistic(OBJECT_PUT_BYTES, bytes); + decrementGauge(OBJECT_PUT_BYTES_PENDING, bytes); } + incrementStatistic(OBJECT_PUT_REQUESTS_COMPLETED); + decrementGauge(OBJECT_PUT_REQUESTS_ACTIVE, 1); } /** @@ -1663,6 +1744,7 @@ public String toString() { sb.append(", workingDir=").append(workingDir); sb.append(", inputPolicy=").append(inputPolicy); sb.append(", partSize=").append(partSize); + sb.append(", fastUpload=").append(fastUploadEnabled); sb.append(", enableMultiObjectsDelete=").append(enableMultiObjectsDelete); sb.append(", maxKeys=").append(maxKeys); if (cannedACL != null) { diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java index b4c406397666e..2f14ffc5c093b 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java @@ -28,6 +28,8 @@ import org.apache.hadoop.metrics2.lib.MutableCounterLong; import org.apache.hadoop.metrics2.lib.MutableGaugeLong; import org.apache.hadoop.metrics2.lib.MutableMetric; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.net.URI; import java.util.HashMap; @@ -50,6 +52,9 @@ @InterfaceAudience.Private @InterfaceStability.Evolving public class S3AInstrumentation { + private static final Logger LOG = LoggerFactory.getLogger( + S3AInstrumentation.class); + public static final String CONTEXT = "S3AFileSystem"; private final MetricsRegistry registry = new MetricsRegistry("S3AFileSystem").setContext(CONTEXT); @@ -99,9 +104,15 @@ public class S3AInstrumentation { OBJECT_METADATA_REQUESTS, OBJECT_MULTIPART_UPLOAD_ABORTED, OBJECT_PUT_BYTES, - OBJECT_PUT_REQUESTS + OBJECT_PUT_REQUESTS, + OBJECT_PUT_REQUESTS_COMPLETED }; + private static final Statistic[] GAUGES_TO_CREATE = { + OBJECT_PUT_REQUESTS_ACTIVE, + OBJECT_PUT_BYTES_PENDING + }; + public S3AInstrumentation(URI name) { UUID fileSystemInstanceId = UUID.randomUUID(); registry.tag("FileSystemId", @@ -141,6 +152,9 @@ public S3AInstrumentation(URI name) { for (Statistic statistic : COUNTERS_TO_CREATE) { counter(statistic); } + for (Statistic statistic : GAUGES_TO_CREATE) { + gauge(statistic.getSymbol(), statistic.getDescription()); + } } /** @@ -252,13 +266,13 @@ public long getCounterValue(String name) { * Lookup a counter by name. Return null if it is not known. * @param name counter name * @return the counter + * @throws IllegalStateException if the metric is not a counter */ private MutableCounterLong lookupCounter(String name) { MutableMetric metric = lookupMetric(name); if (metric == null) { return null; } - Preconditions.checkNotNull(metric, "not found: " + name); if (!(metric instanceof MutableCounterLong)) { throw new IllegalStateException("Metric " + name + " is not a MutableCounterLong: " + metric); @@ -266,6 +280,20 @@ private MutableCounterLong lookupCounter(String name) { return (MutableCounterLong) metric; } + /** + * Look up a gauge. + * @param name gauge name + * @return the gauge or null + * @throws ClassCastException if the metric is not a Gauge. + */ + public MutableGaugeLong lookupGauge(String name) { + MutableMetric metric = lookupMetric(name); + if (metric == null) { + LOG.debug("No metric {}", name); + } + return (MutableGaugeLong) metric; + } + /** * Look up a metric from both the registered set and the lighter weight * stream entries. @@ -340,6 +368,38 @@ public void incrementCounter(Statistic op, long count) { } } + /** + * Increment a specific gauge. + * No-op if not defined. + * @param op operation + * @param count increment value + * @throws ClassCastException if the metric is of the wrong type + */ + public void incrementGauge(Statistic op, long count) { + MutableGaugeLong gauge = lookupGauge(op.getSymbol()); + if (gauge != null) { + gauge.incr(count); + } else { + LOG.debug("No Gauge: "+ op); + } + } + + /** + * Decrement a specific gauge. + * No-op if not defined. + * @param op operation + * @param count increment value + * @throws ClassCastException if the metric is of the wrong type + */ + public void decrementGauge(Statistic op, long count) { + MutableGaugeLong gauge = lookupGauge(op.getSymbol()); + if (gauge != null) { + gauge.decr(count); + } else { + LOG.debug("No Gauge: " + op); + } + } + /** * Create a stream input statistics instance. * @return the new instance diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AOutputStream.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AOutputStream.java index bd9243c2bc8d6..347b4060b4cab 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AOutputStream.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AOutputStream.java @@ -50,8 +50,6 @@ public class S3AOutputStream extends OutputStream { private boolean closed; private String key; private Progressable progress; - private long partSize; - private long partSizeThreshold; private S3AFileSystem fs; private LocalDirAllocator lDirAlloc; @@ -64,9 +62,6 @@ public S3AOutputStream(Configuration conf, this.progress = progress; this.fs = fs; - partSize = fs.getPartitionSize(); - partSizeThreshold = fs.getMultiPartThreshold(); - if (conf.get(BUFFER_DIR, null) != null) { lDirAlloc = new LocalDirAllocator(BUFFER_DIR); } else { @@ -108,9 +103,6 @@ public synchronized void close() throws IOException { backupStream.close(); LOG.debug("OutputStream for key '{}' closed. Now beginning upload", key); - LOG.debug("Minimum upload part size: {} threshold {}" , partSize, - partSizeThreshold); - try { final ObjectMetadata om = fs.newObjectMetadata(); diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java index a5e8e7a1b6a02..1ba074b96640e 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java @@ -213,6 +213,18 @@ public static String stringify(AmazonS3Exception e) { return builder.toString(); } + /** + * Extract the error code of an AmazonServiceException. + * @param e any AmazonClientException + * @return the error code from the exception, or "" if it is the wrong + * type. + */ + public static String extractErrorCode(AmazonClientException e) { + return e instanceof AmazonServiceException ? + ((AmazonServiceException) e).getErrorCode() + : ""; + } + /** * Create a files status instance from a listing. * @param keyPath path to entry diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Statistic.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Statistic.java index 28b61b82029e1..75ce63303dec4 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Statistic.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Statistic.java @@ -77,7 +77,13 @@ public enum Statistic { "Object multipart upload aborted"), OBJECT_PUT_REQUESTS("object_put_requests", "Object put/multipart upload count"), + OBJECT_PUT_REQUESTS_COMPLETED("object_put_requests_completed", + "Object put/multipart upload completed count"), + OBJECT_PUT_REQUESTS_ACTIVE("object_put_requests_active", + "Current number of active put requests"), OBJECT_PUT_BYTES("object_put_bytes", "number of bytes uploaded"), + OBJECT_PUT_BYTES_PENDING("object_put_bytes_pending", + "number of bytes queued for upload/being actively uploaded"), STREAM_ABORTED("stream_aborted", "Count of times the TCP stream was aborted"), STREAM_BACKWARD_SEEK_OPERATIONS("stream_backward_seek_operations", diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md index ce0a19b30cb03..2e326d899551e 100644 --- a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md +++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md @@ -1256,6 +1256,67 @@ 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 + +This happens when using the Fast output stream, `fs.s3a.fast.upload=true` and +the thread pool runs out of capacity. + +``` + [s3a-transfer-shared-pool1-t20] INFO http.AmazonHttpClient (AmazonHttpClient.java:executeHelper(496)) - Unable to execute HTTP request: Timeout waiting for connection from poolorg.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:1025) + at org.apache.hadoop.fs.s3a.S3AFastOutputStream$MultiPartUpload$1.call(S3AFastOutputStream.java:360) + at org.apache.hadoop.fs.s3a.S3AFastOutputStream$MultiPartUpload$1.call(S3AFastOutputStream.java:355) + 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) +``` + +Make sure that `fs.s3a.connection.maximum` is at least larger +than `fs.s3a.threads.max`. + +```xml + + fs.s3a.threads.max + 20 + + + + fs.s3a.connection.maximum + 30 + +``` + +## "Timeout waiting for connection from pool" when reading from S3A + +This happens when more threads are trying to read from an S3A system than +the maximum number of allocated HTTP connections. + +Set `fs.s3a.connection.maximum` to a larger value (and at least as large as +`fs.s3a.threads.max`) + +### Out of heap memory when writing to S3A + + ## Visible S3 Inconsistency Amazon S3 is *an eventually consistent object store*. That is: not a filesystem. diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFastOutputStream.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFastOutputStream.java index b5fa1c3615013..e3092268b4025 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFastOutputStream.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFastOutputStream.java @@ -37,7 +37,6 @@ public class ITestS3AFastOutputStream { private FileSystem fs; - @Rule public Timeout testTimeout = new Timeout(30 * 60 * 1000); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/fileContext/ITestS3AFileContextStatistics.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/fileContext/ITestS3AFileContextStatistics.java index 5e88aba615f82..e1aef757866a0 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/fileContext/ITestS3AFileContextStatistics.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/fileContext/ITestS3AFileContextStatistics.java @@ -34,6 +34,7 @@ public void setUp() throws Exception { fc = S3ATestUtils.createTestFileContext(conf); fc.mkdir(fileContextTestHelper.getTestRootPath(fc, "test"), FileContext.DEFAULT_PERM, true); + FileContext.clearStatistics(); } @After diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/S3AScaleTestBase.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/S3AScaleTestBase.java index b1ab050b9d63b..c39f5d51a3813 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/S3AScaleTestBase.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/S3AScaleTestBase.java @@ -28,6 +28,8 @@ import org.apache.hadoop.fs.s3a.S3AInstrumentation; import org.apache.hadoop.fs.s3a.S3ATestConstants; import org.apache.hadoop.fs.s3a.S3ATestUtils; +import org.apache.hadoop.fs.s3a.Statistic; +import org.apache.hadoop.metrics2.lib.MutableGaugeLong; import org.junit.After; import org.junit.Assert; import org.junit.Before; @@ -59,6 +61,7 @@ public void nameThread() { public static final long _1KB = 1024L; public static final long _1MB = _1KB * _1KB; + public static final long _10MB = _1MB * 10; public static final long _1GB = _1KB * _1MB; /** @@ -253,4 +256,17 @@ protected S3AInstrumentation.InputStreamStatistics getInputStreamStatistics( } } + /** + * Get the gauge value of a statistic. Raises an assertion if + * there is no such gauge. + * @param statistic statistic to look up + * @return the value. + */ + public long gaugeValue(Statistic statistic) { + S3AInstrumentation instrumentation = fs.getInstrumentation(); + MutableGaugeLong gauge = instrumentation.lookupGauge(statistic.getSymbol()); + assertNotNull("No gauge " + statistic + + " in " + instrumentation.dump("", " = ", "\n", true), gauge); + return gauge.value(); + } } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/STestS3AHugeFileCreate.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/STestS3AHugeFileCreate.java index 3bd9803da1289..6c61a7b9e35c0 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/STestS3AHugeFileCreate.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/STestS3AHugeFileCreate.java @@ -28,7 +28,9 @@ import org.apache.hadoop.fs.contract.ContractTestUtils; import org.apache.hadoop.fs.s3a.Constants; import org.apache.hadoop.fs.s3a.S3AFileStatus; +import org.apache.hadoop.fs.s3a.S3AInstrumentation; import org.apache.hadoop.fs.s3a.Statistic; +import org.apache.hadoop.metrics2.lib.MutableGaugeLong; import org.apache.hadoop.util.Progressable; import org.junit.Assume; import org.junit.FixMethodOrder; @@ -42,9 +44,7 @@ import java.io.IOException; import static org.apache.hadoop.fs.contract.ContractTestUtils.toHuman; -import static org.apache.hadoop.fs.s3a.Constants.MIN_MULTIPART_THRESHOLD; -import static org.apache.hadoop.fs.s3a.Constants.SOCKET_RECV_BUFFER; -import static org.apache.hadoop.fs.s3a.Constants.SOCKET_SEND_BUFFER; +import static org.apache.hadoop.fs.s3a.Constants.*; /** * Scale test which creates a huge file. @@ -54,8 +54,8 @@ * an ordering based on the numbers. * * Having this ordering allows the tests to assume that the huge file - * exists. Even so: they should all have an assumes() check at the start, - * in case an individual test is executed. + * exists. Even so: they should all have a {@link #assumeHugeFileExists()} + * check at the start, in case an individual test is executed. */ @FixMethodOrder(MethodSorters.NAME_ASCENDING) public class STestS3AHugeFileCreate extends S3AScaleTestBase { @@ -84,10 +84,11 @@ public void tearDown() throws Exception { @Override protected Configuration createConfiguration() { Configuration configuration = super.createConfiguration(); - configuration.setBoolean(Constants.FAST_UPLOAD, true); + configuration.setBoolean(FAST_UPLOAD, true); configuration.setLong(MIN_MULTIPART_THRESHOLD, 5 * _1MB); configuration.setLong(SOCKET_SEND_BUFFER, BLOCKSIZE); configuration.setLong(SOCKET_RECV_BUFFER, BLOCKSIZE); + configuration.set(USER_AGENT_PREFIX, "STestS3AHugeFileCreate"); return configuration; } @@ -126,9 +127,12 @@ public void test_010_CreateHugeFile() throws IOException { StorageStatistics storageStatistics = fs.getStorageStatistics(); String putRequests = Statistic.OBJECT_PUT_REQUESTS.getSymbol(); String putBytes = Statistic.OBJECT_PUT_BYTES.getSymbol(); + Statistic putRequestsActive = Statistic.OBJECT_PUT_REQUESTS_ACTIVE; + Statistic putBytesPending = Statistic.OBJECT_PUT_BYTES_PENDING; ContractTestUtils.NanoTimer timer = new ContractTestUtils.NanoTimer(); + long blocksPer10MB = blocksPerMB * 10; try(FSDataOutputStream out = fs.create(hugefile, true, BLOCKSIZE, @@ -136,18 +140,19 @@ public void test_010_CreateHugeFile() throws IOException { for (long block = 1; block <= blocks; block++) { out.write(data); - if (block % blocksPerMB == 0) { + // every 10 MB, dump something + if (block % blocksPer10MB == 0) { long written = block * BLOCKSIZE; long percentage = written * 100 / filesize; - LOG.info(String.format("[%03d%%] Written %.2f MB out of %.2f MB;" + - " PUT = %d bytes in %d operations", + LOG.info(String.format("[%02d%%] Written %.2f MB out of %.2f MB;" + + " PUT = %d bytes (%d pending) in %d operations (%d active)", percentage, 1.0 * written / _1MB, 1.0 * filesize / _1MB, storageStatistics.getLong(putBytes), - storageStatistics.getLong( - putRequests) - )); + gaugeValue(putBytesPending), + storageStatistics.getLong(putRequests), + gaugeValue(putRequestsActive))); } } // now close the file @@ -174,12 +179,13 @@ public void test_010_CreateHugeFile() throws IOException { toHuman(timer.nanosPerOperation(putRequestCount))); S3AFileStatus status = fs.getFileStatus(hugefile); assertEquals("File size in " + status, filesize, status.getLen()); + assertEquals("active put requests in \n" + fs, 0, gaugeValue(putRequestsActive)); } /** * Progress callback from AWS. Likely to come in on a different thread. */ - private static class ProgressCallback implements Progressable, + private class ProgressCallback implements Progressable, ProgressListener { private int counter = 0; From 9b25371bfbfb0cd89903a5b754550b0f69ffda0b Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Sun, 28 Aug 2016 12:38:40 +0100 Subject: [PATCH 11/20] HADOOP-13531 S3 output stream allocator to round-robin directories --- .../apache/hadoop/fs/s3a/S3AFileSystem.java | 14 ++++++++ .../apache/hadoop/fs/s3a/S3AOutputStream.java | 23 ++++++------- .../site/markdown/tools/hadoop-aws/index.md | 2 +- .../hadoop/fs/s3a/ITestS3AConfiguration.java | 32 +++++++++++++++++++ 4 files changed, 59 insertions(+), 12 deletions(-) diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java index 7043131277054..e62346c5870ec 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java @@ -66,6 +66,7 @@ import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.GlobalStorageStatistics; +import org.apache.hadoop.fs.LocalDirAllocator; import org.apache.hadoop.fs.LocalFileSystem; import org.apache.hadoop.fs.LocatedFileStatus; import org.apache.hadoop.fs.Path; @@ -117,6 +118,7 @@ public class S3AFileSystem extends FileSystem { private ExecutorService threadPoolExecutor; private long multiPartThreshold; public static final Logger LOG = LoggerFactory.getLogger(S3AFileSystem.class); + private LocalDirAllocator directoryAllocator; private CannedAccessControlList cannedACL; private String serverSideEncryptionAlgorithm; private S3AInstrumentation instrumentation; @@ -218,6 +220,10 @@ public StorageStatistics provide() { conf.getTrimmed(INPUT_FADVISE, INPUT_FADV_NORMAL)); fastUploadEnabled = getConf().getBoolean(FAST_UPLOAD, DEFAULT_FAST_UPLOAD); + + String itemName = conf.get(BUFFER_DIR) != null + ? BUFFER_DIR : "hadoop.tmp.dir"; + directoryAllocator = new LocalDirAllocator(itemName); } catch (AmazonClientException e) { throw translateException("initializing ", new Path(name), e); } @@ -343,6 +349,14 @@ public S3AInputPolicy getInputPolicy() { return inputPolicy; } + /** + * Round-robin directory allocator + * @return allocator of directories for output streams. + */ + LocalDirAllocator getDirectoryAllocator() { + return directoryAllocator; + } + /** * Change the input policy for this FS. * @param inputPolicy new policy diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AOutputStream.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AOutputStream.java index 347b4060b4cab..1532c46c82436 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AOutputStream.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AOutputStream.java @@ -45,13 +45,15 @@ @InterfaceAudience.Private @InterfaceStability.Evolving public class S3AOutputStream extends OutputStream { - private OutputStream backupStream; - private File backupFile; + private final OutputStream backupStream; + private final File backupFile; private boolean closed; - private String key; - private Progressable progress; - private S3AFileSystem fs; - private LocalDirAllocator lDirAlloc; + private final String key; + private final Progressable progress; + private final long partSize; + private final long partSizeThreshold; + private final S3AFileSystem fs; + private final LocalDirAllocator lDirAlloc; public static final Logger LOG = S3AFileSystem.LOG; @@ -62,11 +64,10 @@ public S3AOutputStream(Configuration conf, this.progress = progress; this.fs = fs; - if (conf.get(BUFFER_DIR, null) != null) { - lDirAlloc = new LocalDirAllocator(BUFFER_DIR); - } else { - lDirAlloc = new LocalDirAllocator("${hadoop.tmp.dir}/s3a"); - } + partSize = fs.getPartitionSize(); + partSizeThreshold = fs.getMultiPartThreshold(); + + lDirAlloc = fs.getDirectoryAllocator(); backupFile = lDirAlloc.createTmpFileForWrite("output-", LocalDirAllocator.SIZE_UNKNOWN, conf); diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md index 2e326d899551e..41cd398f3a09f 100644 --- a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md +++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md @@ -1694,7 +1694,7 @@ tests or the `it.test` property for integration tests. mvn clean test -Dtest=TestS3AInputPolicies - mvn clean verify -Dit.test=ITestS3AFileContextStatistics + mvn clean verify -Dit.test=ITestS3AFileContextStatistics -Dtest=none mvn clean verify -Dtest=TestS3A* -Dit.test=ITestS3A* diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AConfiguration.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AConfiguration.java index fca8e491da605..0c529a1cc4fb6 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AConfiguration.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AConfiguration.java @@ -25,9 +25,11 @@ import org.apache.commons.lang.StringUtils; import org.apache.commons.lang.reflect.FieldUtils; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.LocalDirAllocator; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.contract.ContractTestUtils; import org.apache.hadoop.fs.s3native.S3xLoginHelper; +import org.apache.hadoop.test.GenericTestUtils; import org.junit.Rule; import org.junit.Test; import org.junit.rules.Timeout; @@ -35,6 +37,7 @@ import org.slf4j.LoggerFactory; import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotEquals; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertTrue; import static org.junit.Assert.assertEquals; @@ -409,6 +412,35 @@ public void testCustomUserAgent() throws Exception { awsConf.getUserAgent()); } + @Test + public void testDirectoryAllocatorDefval() throws Throwable { + conf = new Configuration(); + conf.unset(Constants.BUFFER_DIR); + fs = S3ATestUtils.createTestFileSystem(conf); + LocalDirAllocator allocator = fs.getDirectoryAllocator(); + File tmp = allocator.createTmpFileForWrite("out-", 1024, conf); + assertTrue("not found: " + tmp, tmp.exists()); + tmp.delete(); + } + + @Test + public void testDirectoryAllocatorRR() throws Throwable { + File dir1 = GenericTestUtils.getRandomizedTestDir(); + File dir2 = GenericTestUtils.getRandomizedTestDir(); + dir1.mkdirs(); + dir2.mkdirs(); + conf = new Configuration(); + conf.set(Constants.BUFFER_DIR, dir1 +", "+dir2); + fs = S3ATestUtils.createTestFileSystem(conf); + LocalDirAllocator allocator = fs.getDirectoryAllocator(); + File tmp1 = allocator.createTmpFileForWrite("out-", 1024, conf); + tmp1.delete(); + File tmp2 = allocator.createTmpFileForWrite("out-", 1024, conf); + tmp2.delete(); + assertNotEquals("round robin not working", + tmp1.getParent(), tmp2.getParent()); + } + /** * Reads and returns a field from an object using reflection. If the field * cannot be found, is null, or is not the expected type, then this method From 6733165d080d10aacd8a008262cd1fd9fa9ab6de Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Mon, 5 Sep 2016 18:48:36 +0100 Subject: [PATCH 12/20] HADOOP-13560 WiP: adding new incremental output stream --- .../hadoop/fs/s3a/S3AFastOutputStream.java | 5 + .../fs/s3a/S3AIncrementalOutputStream.java | 489 ++++++++++++++++++ .../org/apache/hadoop/fs/s3a/S3AOutput.java | 360 +++++++++++++ 3 files changed, 854 insertions(+) create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AIncrementalOutputStream.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AOutput.java diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFastOutputStream.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFastOutputStream.java index 7b4c9149420b0..6ad32a919fcd5 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFastOutputStream.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFastOutputStream.java @@ -33,6 +33,7 @@ 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; @@ -46,7 +47,10 @@ import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; +import java.io.Closeable; +import java.io.File; import java.io.IOException; +import java.io.InputStream; import java.io.OutputStream; import java.util.ArrayList; import java.util.List; @@ -482,4 +486,5 @@ public void progressChanged(ProgressEvent progressEvent) { } } } + } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AIncrementalOutputStream.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AIncrementalOutputStream.java new file mode 100644 index 0000000000000..e32a2862e670d --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AIncrementalOutputStream.java @@ -0,0 +1,489 @@ +/** + * 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 com.amazonaws.AmazonClientException; +import com.amazonaws.event.ProgressEvent; +import com.amazonaws.event.ProgressListener; +import com.amazonaws.services.s3.AmazonS3Client; +import com.amazonaws.services.s3.model.AbortMultipartUploadRequest; +import com.amazonaws.services.s3.model.CannedAccessControlList; +import com.amazonaws.services.s3.model.CompleteMultipartUploadRequest; +import com.amazonaws.services.s3.model.CompleteMultipartUploadResult; +import com.amazonaws.services.s3.model.InitiateMultipartUploadRequest; +import com.amazonaws.services.s3.model.ObjectMetadata; +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.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.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.io.retry.RetryPolicies; +import org.apache.hadoop.io.retry.RetryPolicy; +import org.apache.hadoop.util.Progressable; +import org.slf4j.Logger; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +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 static org.apache.hadoop.fs.s3a.S3AUtils.extractException; +import static org.apache.hadoop.fs.s3a.S3AUtils.translateException; +import static org.apache.hadoop.fs.s3a.Statistic.IGNORED_ERRORS; +import static org.apache.hadoop.fs.s3a.Statistic.OBJECT_MULTIPART_UPLOAD_ABORTED; + +/** + * Upload files/parts asap directly via different buffering mechanisms: + * memory, disk. + *

+ * Uploads are managed low-level rather than through the AWS TransferManager. + * This allows for uploading each part of a multi-part upload as soon as + * the bytes are in memory, rather than waiting until the file is closed. + *

+ * Unstable: statistics and error handling might evolve + */ +@InterfaceAudience.Private +@InterfaceStability.Unstable +public class S3AIncrementalOutputStream extends OutputStream { + + private static final Logger LOG = S3AFileSystem.LOG; + private final S3AFileSystem fs; + private final String key; + private final String bucket; + private final int partitionSize; + private final CannedAccessControlList cannedACL; + private final ProgressListener progressListener; + private final ListeningExecutorService executorService; + private MultiPartUpload multiPartUpload; + private boolean closed; + private S3AOutput.StreamDestination dest; + private int bufferLimit; + private final RetryPolicy retryPolicy; + private final S3AOutput.StreamDestinationFactory destFactory; + private final byte[] singleCharWrite = new byte[1]; + + + /** + * Creates a fast OutputStream that uploads to S3 from memory. + * For MultiPartUploads, as soon as sufficient bytes have been written to + * the stream a part is uploaded immediately (by using the low-level + * multi-part upload API on the AmazonS3Client). + * + * + * @param fs S3AFilesystem + * @param bucket S3 bucket name + * @param key S3 key name + * @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 cannedACL used CannedAccessControlList + * @param partitionSize size of a single part in a multi-part upload (except + * last part) + * @param multiPartThreshold files at least this size use multi-part upload + * @param threadPoolExecutor thread factory + * @param destinationFactory factory for creating stream destinations + * @throws IOException on any problem + */ + public S3AIncrementalOutputStream(S3AFileSystem fs, + String bucket, + String key, + Progressable progress, + CannedAccessControlList cannedACL, + int partitionSize, + ExecutorService threadPoolExecutor, + S3AOutput.StreamDestinationFactory destinationFactory) + throws IOException { + this.fs = fs; + this.bucket = bucket; + this.key = key; + this.cannedACL = cannedACL; + this.destFactory = destinationFactory; + //Ensure limit as ByteArrayOutputStream size cannot exceed Integer.MAX_VALUE + if (partitionSize > Integer.MAX_VALUE) { + this.partitionSize = Integer.MAX_VALUE; + LOG.warn("s3a: MULTIPART_SIZE capped to ~2.14GB (maximum allowed size " + + "when using 'FAST_UPLOAD = true')"); + } else { + this.partitionSize = partitionSize; + } + this.closed = false; + int initialBufferSize = this.fs.getConf() + .getInt(Constants.FAST_BUFFER_SIZE, Constants.DEFAULT_FAST_BUFFER_SIZE); + if (initialBufferSize < 0) { + LOG.warn("s3a: FAST_BUFFER_SIZE should be a positive number. Using " + + "default value"); + initialBufferSize = Constants.DEFAULT_FAST_BUFFER_SIZE; + } else if (initialBufferSize > this.bufferLimit) { + LOG.warn("s3a: automatically adjusting FAST_BUFFER_SIZE to not " + + "exceed MIN_MULTIPART_THRESHOLD"); + initialBufferSize = this.bufferLimit; + } + destFactory.init(fs); + maybeCreateDestStream(); + this.executorService = MoreExecutors.listeningDecorator(threadPoolExecutor); + this.multiPartUpload = null; + this.progressListener = (progress instanceof ProgressListener) ? + (ProgressListener) progress + : new ProgressableListener(progress); + this.retryPolicy = RetryPolicies.retryUpToMaximumCountWithProportionalSleep( + 5, 2000, TimeUnit.MILLISECONDS); + LOG.debug("Initialized S3AFastOutputStream for bucket '{}' key '{}' " + + "output to {}", bucket, key, dest); + } + + public void maybeCreateDestStream() throws IOException { + if (dest == null) { + dest = destFactory.create(this.partitionSize); + } + } + + /** + * Check for the filesystem being open. + * @throws IOException if the filesystem is closed. + */ + void checkOpen() throws IOException { + if (closed) { + throw new IOException("Filesystem closed"); + } + } + + /** + * 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 { + + S3AOutput.validateWriteArgs(source, offset, len); + checkOpen(); + if (len == 0) { + return; + } + maybeCreateDestStream(); + int written = dest.write(source, offset, len); + if (written < len) { + // not everything was written. Trigger an upload then + // process the remainder. + uploadBuffer(); + this.write(source, offset + written, len - written); + } + } + + private synchronized void uploadBuffer() throws IOException { + if (multiPartUpload == null) { + multiPartUpload = initiateMultiPartUpload(); + } + multiPartUpload.uploadPartAsync(dest.openForUpload(), partitionSize); + dest.close(); + dest = null; + + { + /* Upload the existing buffer if it exceeds partSize. This possibly + requires multiple parts! */ + final byte[] allBytes = dest.toByteArray(); + dest = null; //earlier gc? + LOG.debug("Total length of initial buffer: {}", allBytes.length); + int processedPos = 0; + while ((multiPartThreshold - processedPos) >= partitionSize) { + LOG.debug("Initial buffer: processing from byte {} to byte {}", + processedPos, (processedPos + partitionSize - 1)); + multiPartUpload.uploadPartAsync(new ByteArrayInputStream(allBytes, + processedPos, partitionSize), partitionSize); + processedPos += partitionSize; + } + //resize and reset stream + bufferLimit = partitionSize; + dest = new ByteArrayOutputStream(bufferLimit); + dest.write(allBytes, processedPos, multiPartThreshold - processedPos); + } else { + //upload next part + multiPartUpload.uploadPartAsync(new ByteArrayInputStream(dest + .toByteArray()), partitionSize); + dest.reset(); + } + } + + /** + * Close the stream. This will not return until the upload is complete + * or the attempt to perform the upload has failed. + * Exceptions raised in this method are indicative that the write has + * failed and data is at risk of being lost. + * @throws IOException on any failure. + */ + @Override + public synchronized void close() throws IOException { + if (closed) { + return; + } + closed = true; + try { + if (multiPartUpload == null) { + putObject(); + } else { + int size = dest.size(); + if (size > 0) { + //send last part + multiPartUpload.uploadPartAsync(new ByteArrayInputStream(dest + .toByteArray()), size); + } + final List partETags = multiPartUpload + .waitForAllPartUploads(); + multiPartUpload.complete(partETags); + } + // This will delete unnecessary fake parent directories + fs.finishedWrite(key); + LOG.debug("Upload complete for bucket '{}' key '{}'", bucket, key); + } finally { + dest = null; + super.close(); + } + } + + /** + * Create the default metadata for a multipart upload operation. + * @return the metadata to use/extend. + */ + private ObjectMetadata createDefaultMetadata() { + return fs.newObjectMetadata(); + } + + private MultiPartUpload initiateMultiPartUpload() throws IOException { + final InitiateMultipartUploadRequest initiateMPURequest = + new InitiateMultipartUploadRequest(bucket, + key, + createDefaultMetadata()); + initiateMPURequest.setCannedACL(cannedACL); + try { + return new MultiPartUpload( + getClient().initiateMultipartUpload(initiateMPURequest).getUploadId()); + } catch (AmazonClientException ace) { + throw translateException("initiate MultiPartUpload", key, ace); + } + } + + private void putObject() throws IOException { + LOG.debug("Executing regular upload for bucket '{}' key '{}'", + bucket, key); + final ObjectMetadata om = createDefaultMetadata(); + final int size = dest.size(); + om.setContentLength(size); + final PutObjectRequest putObjectRequest = + fs.newPutObjectRequest(key, + om, + new ByteArrayInputStream(dest.toByteArray())); + putObjectRequest.setGeneralProgressListener(progressListener); + ListenableFuture putObjectResult = + executorService.submit(new Callable() { + @Override + public PutObjectResult call() throws Exception { + return fs.putObjectDirect(putObjectRequest); + } + }); + //wait for completion + try { + putObjectResult.get(); + } catch (InterruptedException ie) { + LOG.warn("Interrupted object upload: {}", ie, ie); + Thread.currentThread().interrupt(); + } catch (ExecutionException ee) { + throw extractException("regular upload", key, ee); + } + } + + private AmazonS3Client getClient() { + return fs.getAmazonS3Client(); + } + + private class MultiPartUpload { + private final String uploadId; + private final List> partETagsFutures; + + public MultiPartUpload(String uploadId) { + this.uploadId = uploadId; + this.partETagsFutures = new ArrayList>(); + LOG.debug("Initiated multi-part upload for bucket '{}' key '{}' with " + + "id '{}'", bucket, key, uploadId); + } + + private void uploadPartAsync(InputStream inputStream, + int partSize) { + final int currentPartNumber = partETagsFutures.size() + 1; + final UploadPartRequest request = + new UploadPartRequest().withBucketName(bucket).withKey(key) + .withUploadId(uploadId).withInputStream(inputStream) + .withPartNumber(currentPartNumber).withPartSize(partSize); + request.setGeneralProgressListener(progressListener); + ListenableFuture partETagFuture = + executorService.submit(new Callable() { + @Override + public PartETag call() throws Exception { + LOG.debug("Uploading part {} for id '{}'", currentPartNumber, + uploadId); + return fs.uploadPart(request).getPartETag(); + } + }); + partETagsFutures.add(partETagFuture); + } + + private List waitForAllPartUploads() throws IOException { + try { + return Futures.allAsList(partETagsFutures).get(); + } catch (InterruptedException ie) { + LOG.warn("Interrupted partUpload: {}", ie, ie); + Thread.currentThread().interrupt(); + return null; + } catch (ExecutionException ee) { + //there is no way of recovering so abort + //cancel all partUploads + for (ListenableFuture future : partETagsFutures) { + future.cancel(true); + } + //abort multipartupload + this.abort(); + throw extractException("Multi-part upload with id '" + uploadId + "'", + key, ee); + } + } + + /** + * This completes the upload. + * Sometimes it fails; here retries are handled to avoid losing all data + * on a transient failure. + * @param partETags list of partial uploads + * @throws IOException on any problem + */ + private CompleteMultipartUploadResult complete(List partETags) + throws IOException { + int retryCount = 0; + AmazonClientException lastException; + do { + try { + LOG.debug("Completing multi-part upload for key '{}', id '{}'", + key, uploadId); + return getClient().completeMultipartUpload( + new CompleteMultipartUploadRequest(bucket, + key, + uploadId, + partETags)); + } catch (AmazonClientException e) { + lastException = e; + } + } while (shouldRetry(lastException, retryCount++)); + // this point is only reached if the operation failed more than + // the allowed retry count + throw translateException("Completing multi-part upload", key, + lastException); + } + + /** + * Abort a multi-part upload. Retries are attempted on failures. + */ + public void abort() { + int retryCount = 0; + AmazonClientException lastException; + LOG.warn("Aborting multi-part upload with id '{}'", uploadId); + fs.incrementStatistic(OBJECT_MULTIPART_UPLOAD_ABORTED); + do { + try { + LOG.debug("Completing multi-part upload for key '{}', id '{}'", + key, uploadId); + getClient().abortMultipartUpload(new AbortMultipartUploadRequest(bucket, + key, uploadId)); + return; + } catch (AmazonClientException e) { + lastException = e; + } + } while (shouldRetry(lastException, retryCount++)); + // this point is only reached if the operation failed more than + // the allowed retry count + LOG.warn("Unable to abort multipart upload, you may need to purge " + + "uploaded parts: {}", lastException, lastException); + } + + /** + * Predicate to determine whether a failed operation should + * be attempted again. + * If a retry is advised, the exception is automatically logged and + * the filesystem statistic {@link Statistic.IGNORED_ERRORS} incremented. + * @param e exception raised. + * @param retryCount number of retries already attempted + * @return true if another attempt should be made + */ + private boolean shouldRetry(AmazonClientException e, int retryCount) { + try { + boolean retry = retryPolicy.shouldRetry(e, retryCount, 0, true) + == RetryPolicy.RetryAction.RETRY; + if (retry) { + fs.incrementStatistic(IGNORED_ERRORS); + LOG.info("Retrying operation after exception " + e, e); + } + return retry; + } catch (Exception ignored) { + return false; + } + } + + } + + private static class ProgressableListener implements ProgressListener { + private final Progressable progress; + + public ProgressableListener(Progressable progress) { + this.progress = progress; + } + + public void progressChanged(ProgressEvent progressEvent) { + if (progress != null) { + progress.progress(); + } + } + } + +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AOutput.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AOutput.java new file mode 100644 index 0000000000000..53f1c0bb25b99 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AOutput.java @@ -0,0 +1,360 @@ +/* + * 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 com.google.common.base.Preconditions; +import org.apache.hadoop.io.IOUtils; + +import java.io.BufferedOutputStream; +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.Closeable; +import java.io.File; +import java.io.FileInputStream; +import java.io.FileNotFoundException; +import java.io.FileOutputStream; +import java.io.IOException; +import java.io.InputStream; + +import static org.apache.hadoop.fs.s3a.S3AOutput.StreamDestination.DestState.*; + +/** + * Set of classes to support output streaming; kept all together for easier + * management. + */ +class S3AOutput { + + static abstract class StreamDestinationFactory { + + protected S3AFileSystem owner; + + /** + * Bind to the factory owner. + * @param owner owner factory + */ + void init(S3AFileSystem owner) { + this.owner = owner; + } + + /** + * Create a destination. + * @param limit limit of the destination. + * @return a new destination. + */ + abstract StreamDestination create(int limit) throws IOException; + + } + + static abstract class StreamDestination implements Closeable { + + enum DestState {Writing, Upload, Closed} + + private volatile DestState state = Writing; + + protected synchronized void enterState(DestState current, DestState next) { + verifyState(current); + state = next; + } + + protected void verifyState(DestState current) { + Preconditions.checkState(state == current, + "Expected stream state " + current + " -but actual state is " + state + + " in " + this); + } + + DestState getState() { + return state; + } + + /** + * Return the current destination size. + * @return + */ + abstract int currentSize(); + + abstract boolean hasCapacity(long bytes); + + abstract int remainingCapacity(); + + /** + * Write a series of bytes from the buffer, from the offset. + * Returns the number of bytes written: + * @param b + * @param off + * @param len + * @return + * @throws IOException + */ + int write(byte b[], int off, int len) throws IOException { + verifyState(Writing); + return 0; + } + + void flush() throws IOException { + + } + + /** + * Switch to the upload state and return a stream for uploading. + * @return the stream + * @throws IOException + */ + InputStream openForUpload() throws IOException { + 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 + * @throws IOException + */ + protected synchronized boolean enterClosedState() { + if (!state.equals(Closed)) { + state = Closed; + return true; + } else { + return false; + } + } + } + + class UploadStream extends InputStream { + + private final StreamDestination owner; + private final InputStream inner; + + public UploadStream(StreamDestination owner, InputStream inner) { + this.owner = owner; + this.inner = inner; + } + + @Override + public int read() throws IOException { + return 0; + } + + @Override + public void close() throws IOException { + IOUtils.closeStream(inner); + IOUtils.closeStream(owner); + super.close(); + } + } + + /** + * Stream to memory via a {@code ByteArrayOutputStream}. + * + * This was taken from {@link 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 StreamToMemory extends StreamDestination { + private ByteArrayOutputStream buffer; + private int limit; + + public StreamToMemory(int limit) { + this.limit = limit; + buffer = new ByteArrayOutputStream(); + } + + @Override + int currentSize() { + return buffer.size(); + } + + @Override + InputStream openForUpload() throws IOException { + super.openForUpload(); + ByteArrayInputStream bufferData = new ByteArrayInputStream( + buffer.toByteArray()); + buffer.reset(); + buffer = null; + return bufferData; + } + + @Override + public boolean hasCapacity(long bytes) { + return currentSize() + bytes <= limit; + } + + @Override + public int remainingCapacity() { + return limit - currentSize(); + } + + @Override + int write(byte[] b, int off, int len) throws IOException { + super.write(b, off, len); + int written = Math.min(remainingCapacity(), len); + buffer.write(b, off, written); + return written; + } + + @Override + public void close() throws IOException { + buffer = null; + } + + @Override + public String toString() { + final StringBuilder sb = new StringBuilder( + "StreamToMemory{"); + sb.append("state=").append(getState()); + sb.append(", limit=").append(limit); + sb.append('}'); + return sb.toString(); + } + } + + static class StreamToMemoryFactory extends StreamDestinationFactory { + + @Override + void init(S3AFileSystem owner) { + super.init(owner); + } + + @Override + StreamDestination create(int limit) throws IOException { + return new StreamToMemory(limit); + } + } + + static class StreamToDiskFactory extends StreamDestinationFactory { + + @Override + void init(S3AFileSystem owner) { + super.init(owner); + } + + @Override + StreamDestination create(int limit) throws IOException { + File destFile = owner.getDirectoryAllocator() + .createTmpFileForWrite("s3a", limit, owner.getConf()); + return new StreamToFile(destFile, limit); + } + } + + /** + * Stream to a file. + * This will stop at the limit; the caller is expected to create a new file + */ + static class StreamToFile extends StreamDestination { + private File destFile; + + private int limit; + protected int bytesWritten; + + private BufferedOutputStream out; + private FileInputStream uploadStream; + + public StreamToFile(File destFile, int limit) + throws FileNotFoundException { + this.limit = limit; + this.destFile = destFile; + out = new BufferedOutputStream( + new FileOutputStream(destFile)); + } + + @Override + int currentSize() { + return bytesWritten; + } + + @Override + boolean hasCapacity(long bytes) { + return currentSize() + bytes <= limit; + } + + @Override + int remainingCapacity() { + return limit - bytesWritten; + } + + @Override + int write(byte[] b, int off, int len) throws IOException { + super.write(b, off, len); + int written = Math.min(remainingCapacity(), len); + out.write(b, off, written); + bytesWritten += written; + return written; + } + + @Override + InputStream openForUpload() throws IOException { + super.openForUpload(); + try { + out.flush(); + } finally { + out.close(); + out = null; + } + uploadStream = new FileInputStream(destFile); + return uploadStream; + } + + @Override + public synchronized void close() throws IOException { + enterClosedState(); + IOUtils.closeStream(out); + IOUtils.closeStream(uploadStream); + out = null; + uploadStream = null; + destFile.delete(); + } + + @Override + void flush() throws IOException { + verifyState(Writing); + out.flush(); + } + + @Override + public String toString() { + final StringBuilder sb = new StringBuilder( + "StreamToFile{"); + sb.append("destFile=").append(destFile); + sb.append(", state=").append(getState()); + sb.append(", limit=").append(limit); + sb.append(", bytesWritten=").append(bytesWritten); + sb.append('}'); + return sb.toString(); + } + } + + /** + * Validate args to write command. + * @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 + */ + public 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(); + } + } +} From 9a5dee049bd65af78289815763ff52c770e09aa5 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Tue, 6 Sep 2016 14:46:50 +0100 Subject: [PATCH 13/20] HADOOP-13560 data block design is coalescing and memory buffer writes are passing tests --- .../org/apache/hadoop/fs/s3a/Constants.java | 42 +++ ...tStream.java => S3ABlockOutputStream.java} | 247 +++++++-------- .../{S3AOutput.java => S3ADataBlocks.java} | 288 ++++++++++++------ .../hadoop/fs/s3a/S3AFastOutputStream.java | 2 +- .../apache/hadoop/fs/s3a/S3AFileSystem.java | 99 ++++-- .../org/apache/hadoop/fs/s3a/S3AUtils.java | 30 ++ .../hadoop/fs/s3a/AbstractS3ATestBase.java | 1 + .../fs/s3a/ITestS3ABlockOutputMemory.java | 79 +++++ 8 files changed, 552 insertions(+), 236 deletions(-) rename hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/{S3AIncrementalOutputStream.java => S3ABlockOutputStream.java} (70%) rename hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/{S3AOutput.java => S3ADataBlocks.java} (57%) create mode 100644 hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ABlockOutputMemory.java diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java index cf97c35ba45a9..1cefb73c1701d 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java @@ -32,6 +32,8 @@ @InterfaceStability.Evolving public final class Constants { + public static final int MULTIPART_MIN_SIZE = 5 * 1024 * 1024; + private Constants() { } @@ -178,6 +180,46 @@ private Constants() { public static final String READAHEAD_RANGE = "fs.s3a.readahead.range"; public static final long DEFAULT_READAHEAD_RANGE = 64 * 1024; + /** + * Whether to use the experimental block output mechanism. + * Value: {@value} + */ + @InterfaceStability.Unstable + public static final String BLOCK_OUTPUT = + "fs.s3a.block.output"; + /** + * What buffer to use. + * Default is {@link #OUTPUT_INCREMENTAL_BUFFER_DISK} + * Value: {@value} + */ + @InterfaceStability.Unstable + public static final String BLOCK_OUTPUT_BUFFER = + "fs.s3a.block.output.buffer"; + + /** + * Buffer blocks to disk: {@value}. + * Capacity is limited to available disk space. + */ + + @InterfaceStability.Unstable + public static final String BLOCK_OUTPUT_BUFFER_DISK = + "disk"; + + /** + * Use an in-memory array. Fast but will run of heap rapidly: {@value}. + */ + @InterfaceStability.Unstable + public static final String BLOCK_OUTPUT_BUFFER_ARRAY = + "array"; + + /** + * Default buffer option: {@value}. + */ + + @InterfaceStability.Unstable + public static final String DEFAULT_BLOCK_OUTPUT_BUFFER = + BLOCK_OUTPUT_BUFFER_DISK; + /** * Which input strategy to use for buffering, seeking and similar when * reading data. diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AIncrementalOutputStream.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java similarity index 70% rename from hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AIncrementalOutputStream.java rename to hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java index e32a2862e670d..4105983cfce8b 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AIncrementalOutputStream.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java @@ -32,19 +32,20 @@ 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.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 org.slf4j.Logger; +import org.slf4j.LoggerFactory; -import java.io.ByteArrayInputStream; -import java.io.ByteArrayOutputStream; import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; @@ -52,7 +53,6 @@ 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 static org.apache.hadoop.fs.s3a.S3AUtils.extractException; @@ -61,94 +61,63 @@ import static org.apache.hadoop.fs.s3a.Statistic.OBJECT_MULTIPART_UPLOAD_ABORTED; /** - * Upload files/parts asap directly via different buffering mechanisms: - * memory, disk. - *

- * Uploads are managed low-level rather than through the AWS TransferManager. - * This allows for uploading each part of a multi-part upload as soon as - * the bytes are in memory, rather than waiting until the file is closed. - *

- * Unstable: statistics and error handling might evolve + * 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 -public class S3AIncrementalOutputStream extends OutputStream { +class S3ABlockOutputStream extends OutputStream { - private static final Logger LOG = S3AFileSystem.LOG; + private static final Logger LOG = LoggerFactory.getLogger( + S3ABlockOutputStream.class); private final S3AFileSystem fs; private final String key; private final String bucket; - private final int partitionSize; + private final int blockSize; private final CannedAccessControlList cannedACL; private final ProgressListener progressListener; private final ListeningExecutorService executorService; - private MultiPartUpload multiPartUpload; - private boolean closed; - private S3AOutput.StreamDestination dest; - private int bufferLimit; private final RetryPolicy retryPolicy; - private final S3AOutput.StreamDestinationFactory destFactory; + private final S3ADataBlocks.AbstractBlockFactory blockFactory; private final byte[] singleCharWrite = new byte[1]; - + private MultiPartUpload multiPartUpload; + private volatile boolean closed; + private S3ADataBlocks.DataBlock currentBlock; /** - * Creates a fast OutputStream that uploads to S3 from memory. - * For MultiPartUploads, as soon as sufficient bytes have been written to - * the stream a part is uploaded immediately (by using the low-level - * multi-part upload API on the AmazonS3Client). - * + * S3A output stream which uploads blocks as soon as there is enough + * data. * * @param fs S3AFilesystem - * @param bucket S3 bucket name * @param key S3 key name * @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 cannedACL used CannedAccessControlList - * @param partitionSize size of a single part in a multi-part upload (except - * last part) - * @param multiPartThreshold files at least this size use multi-part upload - * @param threadPoolExecutor thread factory - * @param destinationFactory factory for creating stream destinations + * @param blockSize size of a single block. + * @param blockFactory factory for creating stream destinations * @throws IOException on any problem */ - public S3AIncrementalOutputStream(S3AFileSystem fs, - String bucket, + public S3ABlockOutputStream(S3AFileSystem fs, String key, Progressable progress, - CannedAccessControlList cannedACL, - int partitionSize, - ExecutorService threadPoolExecutor, - S3AOutput.StreamDestinationFactory destinationFactory) + long blockSize, + S3ADataBlocks.AbstractBlockFactory blockFactory) throws IOException { this.fs = fs; - this.bucket = bucket; + this.bucket = fs.getBucket(); this.key = key; - this.cannedACL = cannedACL; - this.destFactory = destinationFactory; - //Ensure limit as ByteArrayOutputStream size cannot exceed Integer.MAX_VALUE - if (partitionSize > Integer.MAX_VALUE) { - this.partitionSize = Integer.MAX_VALUE; - LOG.warn("s3a: MULTIPART_SIZE capped to ~2.14GB (maximum allowed size " + - "when using 'FAST_UPLOAD = true')"); - } else { - this.partitionSize = partitionSize; - } - this.closed = false; - int initialBufferSize = this.fs.getConf() - .getInt(Constants.FAST_BUFFER_SIZE, Constants.DEFAULT_FAST_BUFFER_SIZE); - if (initialBufferSize < 0) { - LOG.warn("s3a: FAST_BUFFER_SIZE should be a positive number. Using " + - "default value"); - initialBufferSize = Constants.DEFAULT_FAST_BUFFER_SIZE; - } else if (initialBufferSize > this.bufferLimit) { - LOG.warn("s3a: automatically adjusting FAST_BUFFER_SIZE to not " + - "exceed MIN_MULTIPART_THRESHOLD"); - initialBufferSize = this.bufferLimit; - } - destFactory.init(fs); - maybeCreateDestStream(); - this.executorService = MoreExecutors.listeningDecorator(threadPoolExecutor); + this.cannedACL = fs.getCannedACL(); + this.blockFactory = blockFactory; + this.blockSize = (int) blockSize; + Preconditions.checkArgument(blockSize >= Constants.MULTIPART_MIN_SIZE, + "Block size is too small: %d", blockSize); + this.executorService = MoreExecutors.listeningDecorator( + fs.getThreadPoolExecutor()); this.multiPartUpload = null; this.progressListener = (progress instanceof ProgressListener) ? (ProgressListener) progress @@ -156,12 +125,13 @@ public S3AIncrementalOutputStream(S3AFileSystem fs, this.retryPolicy = RetryPolicies.retryUpToMaximumCountWithProportionalSleep( 5, 2000, TimeUnit.MILLISECONDS); LOG.debug("Initialized S3AFastOutputStream for bucket '{}' key '{}' " + - "output to {}", bucket, key, dest); + "output to {}", bucket, key, currentBlock); + maybeCreateDestStream(); } public void maybeCreateDestStream() throws IOException { - if (dest == null) { - dest = destFactory.create(this.partitionSize); + if (currentBlock == null) { + currentBlock = blockFactory.create(this.blockSize); } } @@ -201,53 +171,42 @@ public synchronized void write(int b) throws IOException { public synchronized void write(byte[] source, int offset, int len) throws IOException { - S3AOutput.validateWriteArgs(source, offset, len); + S3ADataBlocks.validateWriteArgs(source, offset, len); checkOpen(); if (len == 0) { return; } maybeCreateDestStream(); - int written = dest.write(source, offset, len); + int written = currentBlock.write(source, offset, len); if (written < len) { - // not everything was written. Trigger an upload then - // process the remainder. - uploadBuffer(); + // not everything was written. + // Trigger an upload then process the remainder. + LOG.debug("writing more data than block has capacity"); + uploadCurrentBlock(); + // tail recursion is mildly expensive, but given buffer sizes must be MB. + // it's unlikely to recurse very deeply. this.write(source, offset + written, len - written); + } else { + if (currentBlock.remainingCapacity() == 0) { + // the whole buffer is done, trigger an upload + LOG.debug("Current block is full"); + uploadCurrentBlock(); + } } } - private synchronized void uploadBuffer() throws IOException { + /** + * Trigger the upload. + * @throws IOException Problems opening the destination for upload + * or initializing the upload. + */ + private synchronized void uploadCurrentBlock() throws IOException { if (multiPartUpload == null) { multiPartUpload = initiateMultiPartUpload(); } - multiPartUpload.uploadPartAsync(dest.openForUpload(), partitionSize); - dest.close(); - dest = null; - - { - /* Upload the existing buffer if it exceeds partSize. This possibly - requires multiple parts! */ - final byte[] allBytes = dest.toByteArray(); - dest = null; //earlier gc? - LOG.debug("Total length of initial buffer: {}", allBytes.length); - int processedPos = 0; - while ((multiPartThreshold - processedPos) >= partitionSize) { - LOG.debug("Initial buffer: processing from byte {} to byte {}", - processedPos, (processedPos + partitionSize - 1)); - multiPartUpload.uploadPartAsync(new ByteArrayInputStream(allBytes, - processedPos, partitionSize), partitionSize); - processedPos += partitionSize; - } - //resize and reset stream - bufferLimit = partitionSize; - dest = new ByteArrayOutputStream(bufferLimit); - dest.write(allBytes, processedPos, multiPartThreshold - processedPos); - } else { - //upload next part - multiPartUpload.uploadPartAsync(new ByteArrayInputStream(dest - .toByteArray()), partitionSize); - dest.reset(); - } + multiPartUpload.uploadBlockAsync(currentBlock); + currentBlock.close(); + currentBlock = null; } /** @@ -262,16 +221,22 @@ public synchronized void close() throws IOException { if (closed) { return; } + LOG.debug("Closing {}, data to upload = {}", this, + currentBlock == null ? 0 : currentBlock.dataSize() ); closed = true; + if (currentBlock == null) { + return; + } try { if (multiPartUpload == null) { + // no uploads of data have taken place, put the single block up. putObject(); } else { - int size = dest.size(); - if (size > 0) { + // there has already been at least one block scheduled for upload; + // put up the current then wait + if (currentBlock.hasData()) { //send last part - multiPartUpload.uploadPartAsync(new ByteArrayInputStream(dest - .toByteArray()), size); + multiPartUpload.uploadBlockAsync(currentBlock); } final List partETags = multiPartUpload .waitForAllPartUploads(); @@ -281,8 +246,10 @@ public synchronized void close() throws IOException { fs.finishedWrite(key); LOG.debug("Upload complete for bucket '{}' key '{}'", bucket, key); } finally { - dest = null; - super.close(); + LOG.debug("Closing block and factory"); + IOUtils.closeStream(currentBlock); + IOUtils.closeStream(blockFactory); + currentBlock = null; } } @@ -295,6 +262,7 @@ private ObjectMetadata createDefaultMetadata() { } private MultiPartUpload initiateMultiPartUpload() throws IOException { + LOG.debug("Initiating Multipart upload for block {}", currentBlock); final InitiateMultipartUploadRequest initiateMPURequest = new InitiateMultipartUploadRequest(bucket, key, @@ -308,24 +276,33 @@ private MultiPartUpload initiateMultiPartUpload() throws IOException { } } + /** + * Upload the current block as a single PUT request. + * @throws IOException any problem. + */ private void putObject() throws IOException { LOG.debug("Executing regular upload for bucket '{}' key '{}'", bucket, key); + final ObjectMetadata om = createDefaultMetadata(); - final int size = dest.size(); + final S3ADataBlocks.DataBlock block = this.currentBlock; + final int size = block.dataSize(); om.setContentLength(size); final PutObjectRequest putObjectRequest = fs.newPutObjectRequest(key, om, - new ByteArrayInputStream(dest.toByteArray())); + block.openForUpload()); putObjectRequest.setGeneralProgressListener(progressListener); ListenableFuture putObjectResult = executorService.submit(new Callable() { @Override public PutObjectResult call() throws Exception { - return fs.putObjectDirect(putObjectRequest); + PutObjectResult result = fs.putObjectDirect(putObjectRequest); + block.close(); + return result; } }); + currentBlock = null; //wait for completion try { putObjectResult.get(); @@ -341,32 +318,60 @@ private AmazonS3Client getClient() { return fs.getAmazonS3Client(); } + @Override + public String toString() { + final StringBuilder sb = new StringBuilder( + "S3ABlockOutputStream{"); + sb.append("key='").append(key).append('\''); + sb.append(", bucket='").append(bucket).append('\''); + sb.append(", blockSize=").append(blockSize); + sb.append(", dest=").append(currentBlock); + sb.append('}'); + return sb.toString(); + } + + /** + * Multiple partition upload. + */ private class MultiPartUpload { private final String uploadId; private final List> partETagsFutures; public MultiPartUpload(String uploadId) { this.uploadId = uploadId; - this.partETagsFutures = new ArrayList>(); + this.partETagsFutures = new ArrayList<>(); LOG.debug("Initiated multi-part upload for bucket '{}' key '{}' with " + "id '{}'", bucket, key, uploadId); } - private void uploadPartAsync(InputStream inputStream, - int partSize) { + /** + * Upload a block of data. + * @param block block to upload + * @throws IOException upload failure + */ + private void uploadBlockAsync(final S3ADataBlocks.DataBlock block) + throws IOException { + LOG.debug("Queueing upload of {}", block); + final int size = block.dataSize(); + final InputStream inputStream = block.openForUpload(); final int currentPartNumber = partETagsFutures.size() + 1; final UploadPartRequest request = new UploadPartRequest().withBucketName(bucket).withKey(key) .withUploadId(uploadId).withInputStream(inputStream) - .withPartNumber(currentPartNumber).withPartSize(partSize); + .withPartNumber(currentPartNumber).withPartSize(size); request.setGeneralProgressListener(progressListener); ListenableFuture partETagFuture = executorService.submit(new Callable() { @Override public PartETag call() throws Exception { + // this is the queued upload operation LOG.debug("Uploading part {} for id '{}'", currentPartNumber, uploadId); - return fs.uploadPart(request).getPartETag(); + // do the upload + PartETag partETag = fs.uploadPart(request).getPartETag(); + // close the block, triggering its cleanup + block.close(); + return partETag; } }); partETagsFutures.add(partETagFuture); @@ -393,7 +398,7 @@ private List waitForAllPartUploads() throws IOException { } /** - * This completes the upload. + * This completes a multipart upload. * Sometimes it fails; here retries are handled to avoid losing all data * on a transient failure. * @param partETags list of partial uploads @@ -405,8 +410,8 @@ private CompleteMultipartUploadResult complete(List partETags) AmazonClientException lastException; do { try { - LOG.debug("Completing multi-part upload for key '{}', id '{}'", - key, uploadId); + LOG.debug("Completing multi-part upload for key '{}', id '{}'" , + key, uploadId); return getClient().completeMultipartUpload( new CompleteMultipartUploadRequest(bucket, key, @@ -428,11 +433,10 @@ private CompleteMultipartUploadResult complete(List partETags) public void abort() { int retryCount = 0; AmazonClientException lastException; - LOG.warn("Aborting multi-part upload with id '{}'", uploadId); fs.incrementStatistic(OBJECT_MULTIPART_UPLOAD_ABORTED); do { try { - LOG.debug("Completing multi-part upload for key '{}', id '{}'", + LOG.debug("Aborting multi-part upload for key '{}', id '{}'", key, uploadId); getClient().abortMultipartUpload(new AbortMultipartUploadRequest(bucket, key, uploadId)); @@ -451,7 +455,7 @@ public void abort() { * Predicate to determine whether a failed operation should * be attempted again. * If a retry is advised, the exception is automatically logged and - * the filesystem statistic {@link Statistic.IGNORED_ERRORS} incremented. + * the filesystem statistic {@link Statistic#IGNORED_ERRORS} incremented. * @param e exception raised. * @param retryCount number of retries already attempted * @return true if another attempt should be made @@ -470,6 +474,7 @@ private boolean shouldRetry(AmazonClientException e, int retryCount) { } } + } private static class ProgressableListener implements ProgressListener { diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AOutput.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ADataBlocks.java similarity index 57% rename from hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AOutput.java rename to hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ADataBlocks.java index 53f1c0bb25b99..7e5d79320cb4d 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AOutput.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ADataBlocks.java @@ -19,7 +19,8 @@ package org.apache.hadoop.fs.s3a; import com.google.common.base.Preconditions; -import org.apache.hadoop.io.IOUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.io.BufferedOutputStream; import java.io.ByteArrayInputStream; @@ -32,15 +33,17 @@ import java.io.IOException; import java.io.InputStream; -import static org.apache.hadoop.fs.s3a.S3AOutput.StreamDestination.DestState.*; +import static org.apache.hadoop.fs.s3a.S3ADataBlocks.DataBlock.DestState.*; /** - * Set of classes to support output streaming; kept all together for easier - * management. + * Set of classes to support output streaming into blocks which are then + * uploaded as partitions. */ -class S3AOutput { +class S3ADataBlocks { + static final Logger LOG = LoggerFactory.getLogger(S3ADataBlocks.class); + ; - static abstract class StreamDestinationFactory { + static abstract class AbstractBlockFactory implements Closeable { protected S3AFileSystem owner; @@ -53,15 +56,56 @@ void init(S3AFileSystem owner) { } /** - * Create a destination. - * @param limit limit of the destination. - * @return a new destination. + * Create a block. + * @param limit limit of the block. + * @return a new block. */ - abstract StreamDestination create(int limit) throws IOException; + 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 { + } } - static abstract class StreamDestination implements Closeable { + static class MemoryBlockFactory extends AbstractBlockFactory { + + @Override + void init(S3AFileSystem owner) { + super.init(owner); + } + + @Override + DataBlock create(int limit) throws IOException { + return new ByteArrayBlock(limit); + } + + + } + + /** + * Buffer blocks to disk. + */ + static class DiskBlockFactory extends AbstractBlockFactory { + + @Override + void init(S3AFileSystem owner) { + super.init(owner); + } + + @Override + DataBlock create(int limit) throws IOException { + File destFile = owner.getDirectoryAllocator() + .createTmpFileForWrite("s3ablock", limit, owner.getConf()); + return new FileBlock(destFile, limit); + } + } + + static abstract class DataBlock implements Closeable { enum DestState {Writing, Upload, Closed} @@ -69,6 +113,7 @@ enum DestState {Writing, Upload, Closed} protected synchronized void enterState(DestState current, DestState next) { verifyState(current); + LOG.debug("{}: entering state {}" , this, next); state = next; } @@ -83,13 +128,21 @@ DestState getState() { } /** - * Return the current destination size. - * @return + * Return the current data size. + * @return the size of the data */ - abstract int currentSize(); + abstract int dataSize(); abstract boolean hasCapacity(long bytes); + /** + * Is there data in the block? + * @return + */ + boolean hasData() { + return dataSize() > 0; + } + abstract int remainingCapacity(); /** @@ -98,7 +151,7 @@ DestState getState() { * @param b * @param off * @param len - * @return + * @return number of bytes written * @throws IOException */ int write(byte b[], int off, int len) throws IOException { @@ -128,7 +181,7 @@ InputStream openForUpload() throws IOException { */ protected synchronized boolean enterClosedState() { if (!state.equals(Closed)) { - state = Closed; + enterState(state, Closed); return true; } else { return false; @@ -136,29 +189,6 @@ protected synchronized boolean enterClosedState() { } } - class UploadStream extends InputStream { - - private final StreamDestination owner; - private final InputStream inner; - - public UploadStream(StreamDestination owner, InputStream inner) { - this.owner = owner; - this.inner = inner; - } - - @Override - public int read() throws IOException { - return 0; - } - - @Override - public void close() throws IOException { - IOUtils.closeStream(inner); - IOUtils.closeStream(owner); - super.close(); - } - } - /** * Stream to memory via a {@code ByteArrayOutputStream}. * @@ -168,23 +198,30 @@ public void close() throws IOException { * the JVM-wide upload bandwidth to the S3 endpoint. */ - static class StreamToMemory extends StreamDestination { + static class ByteArrayBlock extends DataBlock { private ByteArrayOutputStream buffer; private int limit; + // cache data size so that it is consistent after the buffer is reset. + private Integer dataSize; - public StreamToMemory(int limit) { + public 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 currentSize() { - return buffer.size(); + int dataSize() { + return dataSize != null ? dataSize : buffer.size(); } @Override InputStream openForUpload() throws IOException { super.openForUpload(); + dataSize = buffer.size(); ByteArrayInputStream bufferData = new ByteArrayInputStream( buffer.toByteArray()); buffer.reset(); @@ -194,12 +231,12 @@ InputStream openForUpload() throws IOException { @Override public boolean hasCapacity(long bytes) { - return currentSize() + bytes <= limit; + return dataSize() + bytes <= limit; } @Override public int remainingCapacity() { - return limit - currentSize(); + return limit - dataSize(); } @Override @@ -212,54 +249,31 @@ int write(byte[] b, int off, int len) throws IOException { @Override public void close() throws IOException { - buffer = null; + if (enterClosedState()) { + LOG.debug("Closed {}", this); + buffer = null; + } } @Override public String toString() { final StringBuilder sb = new StringBuilder( - "StreamToMemory{"); + "ByteArrayBlock{"); sb.append("state=").append(getState()); + sb.append(", dataSize=").append(dataSize()); sb.append(", limit=").append(limit); sb.append('}'); return sb.toString(); } } - static class StreamToMemoryFactory extends StreamDestinationFactory { - - @Override - void init(S3AFileSystem owner) { - super.init(owner); - } - - @Override - StreamDestination create(int limit) throws IOException { - return new StreamToMemory(limit); - } - } - - static class StreamToDiskFactory extends StreamDestinationFactory { - - @Override - void init(S3AFileSystem owner) { - super.init(owner); - } - - @Override - StreamDestination create(int limit) throws IOException { - File destFile = owner.getDirectoryAllocator() - .createTmpFileForWrite("s3a", limit, owner.getConf()); - return new StreamToFile(destFile, limit); - } - } - /** * Stream to a file. - * This will stop at the limit; the caller is expected to create a new file + * This will stop at the limit; the caller is expected to create a new block */ - static class StreamToFile extends StreamDestination { - private File destFile; + static class FileBlock extends DataBlock { + + private File bufferFile; private int limit; protected int bytesWritten; @@ -267,22 +281,22 @@ static class StreamToFile extends StreamDestination { private BufferedOutputStream out; private FileInputStream uploadStream; - public StreamToFile(File destFile, int limit) + public FileBlock(File bufferFile, int limit) throws FileNotFoundException { this.limit = limit; - this.destFile = destFile; + this.bufferFile = bufferFile; out = new BufferedOutputStream( - new FileOutputStream(destFile)); + new FileOutputStream(bufferFile)); } @Override - int currentSize() { + int dataSize() { return bytesWritten; } @Override boolean hasCapacity(long bytes) { - return currentSize() + bytes <= limit; + return dataSize() + bytes <= limit; } @Override @@ -308,18 +322,26 @@ InputStream openForUpload() throws IOException { out.close(); out = null; } - uploadStream = new FileInputStream(destFile); - return uploadStream; + uploadStream = new FileInputStream(bufferFile); + return new FileDeletingInputStream(uploadStream); } @Override public synchronized void close() throws IOException { enterClosedState(); + LOG.debug("Closed {}", this); + if (bufferFile.exists()) { + LOG.warn("Buffer file {} exists —close upload stream", bufferFile); + } +/* IOUtils.closeStream(out); - IOUtils.closeStream(uploadStream); out = null; - uploadStream = null; - destFile.delete(); + if (uploadStream != null) { + IOUtils.closeStream(uploadStream); + } else { + destFile.delete(); + } +*/ } @Override @@ -331,14 +353,78 @@ void flush() throws IOException { @Override public String toString() { final StringBuilder sb = new StringBuilder( - "StreamToFile{"); - sb.append("destFile=").append(destFile); + "FileBlock{"); + sb.append("destFile=").append(bufferFile); sb.append(", state=").append(getState()); + sb.append(", dataSize=").append(dataSize()); sb.append(", limit=").append(limit); - sb.append(", bytesWritten=").append(bytesWritten); sb.append('}'); return sb.toString(); } + + class FileDeletingInputStream extends ForwardingInputStream { + + public FileDeletingInputStream(InputStream source) { + super(source); + } + + @Override + public void close() throws IOException { + super.close(); + bufferFile.delete(); + } + } + } + + /** + * Stream which forwards everything to its inner class. + * For ease of subclassing. + */ + static class ForwardingInputStream extends InputStream { + + protected final InputStream source; + + public ForwardingInputStream(InputStream source) { + this.source = source; + } + + public int read() throws IOException { + return source.read(); + } + + public int read(byte[] b) throws IOException { + return source.read(b); + } + + public int read(byte[] b, int off, int len) throws IOException { + return source.read(b, off, len); + } + + public long skip(long n) throws IOException { + return source.skip(n); + } + + public int available() throws IOException { + return source.available(); + } + + public void close() throws IOException { + LOG.debug("Closing inner stream"); + source.close(); + } + + public void mark(int readlimit) { + source.mark(readlimit); + } + + public void reset() throws IOException { + source.reset(); + } + + public boolean markSupported() { + return source.markSupported(); + } + } /** @@ -349,7 +435,7 @@ public String toString() { * @throws NullPointerException for a null buffer * @throws IndexOutOfBoundsException if indices are out of range */ - public static void validateWriteArgs(byte[] b, int off, int len) + static void validateWriteArgs(byte[] b, int off, int len) throws IOException { Preconditions.checkNotNull(b); if ((off < 0) || (off > b.length) || (len < 0) || @@ -357,4 +443,22 @@ public static void validateWriteArgs(byte[] b, int off, int len) throw new IndexOutOfBoundsException(); } } + + /** + * Create a factory. + * @param name factory name -the option from {@link Constants}. + * @return the factory, ready to be initialized. + * @throws IllegalArgumentException if the name is unknown. + */ + static AbstractBlockFactory createFactory(String name) { + switch (name) { + case Constants.BLOCK_OUTPUT_BUFFER_ARRAY: + return new MemoryBlockFactory(); + case Constants.BLOCK_OUTPUT_BUFFER_DISK: + return new DiskBlockFactory(); + default: + throw new IllegalArgumentException("Unsupported block buffer" + + " \"" + name + "\""); + } + } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFastOutputStream.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFastOutputStream.java index 6ad32a919fcd5..60a8fb74e46b4 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFastOutputStream.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFastOutputStream.java @@ -452,7 +452,7 @@ public void abort() { * Predicate to determine whether a failed operation should * be attempted again. * If a retry is advised, the exception is automatically logged and - * the filesystem statistic {@link Statistic.IGNORED_ERRORS} incremented. + * the filesystem statistic {@link Statistic#IGNORED_ERRORS} incremented. * @param e exception raised. * @param retryCount number of retries already attempted * @return true if another attempt should be made diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java index e62346c5870ec..b35bd38c159f0 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java @@ -131,6 +131,10 @@ public class S3AFileSystem extends FileSystem { // 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 fastUploadEnabled; + private boolean blockUploadEnabled; + private String blockOutputBuffer; + private S3ADataBlocks.AbstractBlockFactory + blockFactory; /** Called after a new FileSystem instance is constructed. * @param name a uri whose authority section names the host, port, etc. @@ -157,18 +161,10 @@ public void initialize(URI name, Configuration conf) throws IOException { maxKeys = intOption(conf, MAX_PAGING_KEYS, DEFAULT_MAX_PAGING_KEYS, 1); listing = new Listing(this); - partSize = conf.getLong(MULTIPART_SIZE, DEFAULT_MULTIPART_SIZE); - if (partSize < 5 * 1024 * 1024) { - LOG.error(MULTIPART_SIZE + " must be at least 5 MB"); - partSize = 5 * 1024 * 1024; - } - - multiPartThreshold = conf.getLong(MIN_MULTIPART_THRESHOLD, + partSize = getSizeProperty(conf, MULTIPART_SIZE, DEFAULT_MULTIPART_SIZE); + multiPartThreshold = getSizeProperty(conf, MIN_MULTIPART_THRESHOLD, DEFAULT_MIN_MULTIPART_THRESHOLD); - if (multiPartThreshold < 5 * 1024 * 1024) { - LOG.error(MIN_MULTIPART_THRESHOLD + " must be at least 5 MB"); - multiPartThreshold = 5 * 1024 * 1024; - } + //check but do not store the block size longOption(conf, FS_S3A_BLOCK_SIZE, DEFAULT_BLOCKSIZE, 1); enableMultiObjectsDelete = conf.getBoolean(ENABLE_MULTI_DELETE, true); @@ -213,6 +209,9 @@ public StorageStatistics provide() { verifyBucketExists(); initMultipartUploads(conf); + String bufferDir = conf.get(BUFFER_DIR) != null + ? BUFFER_DIR : "hadoop.tmp.dir"; + directoryAllocator = new LocalDirAllocator(bufferDir); serverSideEncryptionAlgorithm = conf.getTrimmed(SERVER_SIDE_ENCRYPTION_ALGORITHM); @@ -224,6 +223,23 @@ public StorageStatistics provide() { String itemName = conf.get(BUFFER_DIR) != null ? BUFFER_DIR : "hadoop.tmp.dir"; directoryAllocator = new LocalDirAllocator(itemName); + blockUploadEnabled = conf.getBoolean(BLOCK_OUTPUT, false); + if (blockUploadEnabled) { + blockOutputBuffer = conf.getTrimmed(BLOCK_OUTPUT_BUFFER, + DEFAULT_BLOCK_OUTPUT_BUFFER); + partSize = ensureIntVal(MULTIPART_SIZE, partSize); + blockFactory = S3ADataBlocks.createFactory(blockOutputBuffer); + blockFactory.init(this); + LOG.debug("Uploading data via Block Upload, buffer = {}", + blockOutputBuffer); + } else { + fastUploadEnabled = getConf().getBoolean(FAST_UPLOAD, + DEFAULT_FAST_UPLOAD); + if (fastUploadEnabled) { + partSize = ensureIntVal(MULTIPART_SIZE, partSize); + + } + } } catch (AmazonClientException e) { throw translateException("initializing ", new Path(name), e); } @@ -357,6 +373,30 @@ LocalDirAllocator getDirectoryAllocator() { return directoryAllocator; } + /** + * Get the bucket of this filesystem + * @return the bucket + */ + public String getBucket() { + return bucket; + } + + /** + * Get the thread pool of this FS + * @return the (blocking) thread pool. + */ + public ExecutorService getThreadPoolExecutor() { + return threadPoolExecutor; + } + + /** + * ACL list if set. + * @return an ACL list or null. + */ + public CannedAccessControlList getCannedACL() { + return cannedACL; + } + /** * Change the input policy for this FS. * @param inputPolicy new policy @@ -505,8 +545,17 @@ public FSDataOutputStream create(Path f, FsPermission permission, } instrumentation.fileCreated(); - if (fastUploadEnabled) { - return new FSDataOutputStream( + FSDataOutputStream output; + if (blockUploadEnabled) { + output = new FSDataOutputStream( + new S3ABlockOutputStream(this, + key, + progress, + partSize, + blockFactory), + null); + } else if (fastUploadEnabled) { + output = new FSDataOutputStream( new S3AFastOutputStream(s3, this, bucket, @@ -517,16 +566,19 @@ public FSDataOutputStream create(Path f, FsPermission permission, multiPartThreshold, threadPoolExecutor), statistics); + } else { + + // We pass null to FSDataOutputStream so it won't count writes that + // are being buffered to a file + output = new FSDataOutputStream( + new S3AOutputStream(getConf(), + this, + key, + progress + ), + null); } - // We pass null to FSDataOutputStream so it won't count writes that - // are being buffered to a file - return new FSDataOutputStream( - new S3AOutputStream(getConf(), - this, - key, - progress - ), - null); + return output; } /** @@ -1772,6 +1824,9 @@ public String toString() { .append(serverSideEncryptionAlgorithm) .append('\''); } + if (blockFactory != null) { + sb.append("Block Upload via ").append(blockFactory); + } sb.append(", statistics {") .append(statistics) .append("}"); diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java index 1ba074b96640e..bbbed6a389aa0 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java @@ -48,6 +48,9 @@ import static org.apache.hadoop.fs.s3a.Constants.ACCESS_KEY; import static org.apache.hadoop.fs.s3a.Constants.AWS_CREDENTIALS_PROVIDER; +import static org.apache.hadoop.fs.s3a.Constants.DEFAULT_MIN_MULTIPART_THRESHOLD; +import static org.apache.hadoop.fs.s3a.Constants.MIN_MULTIPART_THRESHOLD; +import static org.apache.hadoop.fs.s3a.Constants.MULTIPART_MIN_SIZE; import static org.apache.hadoop.fs.s3a.Constants.SECRET_KEY; /** @@ -455,4 +458,31 @@ static long longOption(Configuration conf, key, v, min)); return v; } + + public static long getSizeProperty(Configuration conf, + String property, long defVal) { + long partSize = conf.getLong(property, defVal); + if (partSize < MULTIPART_MIN_SIZE) { + LOG.error(property + " must be at least 5 MB"); + partSize = MULTIPART_MIN_SIZE; + } + return partSize; + } + + /** + * Ensure that the long value is in the range of an integer. + * @param name property name for error messages + * @param size original size + * @return the size, guaranteed to be less than or equal to the max + * value of an integer. + */ + public static int ensureIntVal(String name, long size) { + if (size > Integer.MAX_VALUE) { + LOG.warn("s3a: " + name + " capped to ~2.14GB" + + " (maximum allowed size with current output mechanism)"); + return Integer.MAX_VALUE; + } else { + return (int)size; + } + } } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/AbstractS3ATestBase.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/AbstractS3ATestBase.java index b7973b347e725..e049fd16aa328 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/AbstractS3ATestBase.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/AbstractS3ATestBase.java @@ -48,6 +48,7 @@ protected AbstractFSContract createContract(Configuration conf) { @Override public void teardown() throws Exception { super.teardown(); + describe("closing file system"); IOUtils.closeStream(getFileSystem()); } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ABlockOutputMemory.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ABlockOutputMemory.java new file mode 100644 index 0000000000000..c105f506440aa --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ABlockOutputMemory.java @@ -0,0 +1,79 @@ +/** + * 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 org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.contract.ContractTestUtils; +import org.junit.After; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.Timeout; + +import static org.apache.hadoop.fs.s3a.Constants.*; + +import java.io.IOException; + +/** + * Tests regular and multi-part upload functionality for + * {@link S3ABlockOutputStream} with the block buffered in memory. + * + * File sizes are kept small to reduce test duration on slow connections. + */ +public class ITestS3ABlockOutputMemory extends AbstractS3ATestBase { + + @Rule + public Timeout testTimeout = new Timeout(30 * 60 * 1000); + + @Override + protected Configuration createConfiguration() { + Configuration conf = super.createConfiguration(); + S3ATestUtils.disableFilesystemCaching(conf); + conf.setLong(MIN_MULTIPART_THRESHOLD, MULTIPART_MIN_SIZE); + conf.setInt(MULTIPART_SIZE, MULTIPART_MIN_SIZE); + conf.setBoolean(BLOCK_OUTPUT, true); + conf.set(BLOCK_OUTPUT_BUFFER, getBlockOutputBufferName()); + return conf; + } + + protected String getBlockOutputBufferName() { + return BLOCK_OUTPUT_BUFFER_ARRAY; + } + + @Test + public void testRegularUpload() throws IOException { + verifyUpload("regular", 1024); + } + + @Test + public void testMultiPartUpload() throws IOException { + verifyUpload("multipart", MULTIPART_MIN_SIZE + 1024 * 1024); + } + + public void verifyUpload(String name, int fileSize) throws IOException { + Path dest = path(name); + describe(name + " upload to " + dest); + ContractTestUtils.createAndVerifyFile( + getFileSystem(), + dest, + fileSize); + } +} From 2c86048b370ab24b15011101f29503201ba26317 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Tue, 6 Sep 2016 20:58:14 +0100 Subject: [PATCH 14/20] 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. --- .../hadoop/fs/s3a/S3ABlockOutputStream.java | 46 ++++++++++++++----- .../apache/hadoop/fs/s3a/S3ADataBlocks.java | 5 +- .../fs/s3a/ITestS3ABlockOutputDisk.java | 27 +++++++++++ .../fs/s3a/ITestS3ABlockOutputMemory.java | 12 +++-- ...te.java => AbstractSTestS3AHugeFiles.java} | 23 ++-------- .../scale/STestS3AHugeFilesDiskBlocks.java | 40 ++++++++++++++++ .../scale/STestS3AHugeFilesFastUpload.java | 34 ++++++++++++++ .../scale/STestS3AHugeFilesMemoryBlocks.java | 37 +++++++++++++++ 8 files changed, 187 insertions(+), 37 deletions(-) create mode 100644 hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ABlockOutputDisk.java rename hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/{STestS3AHugeFileCreate.java => AbstractSTestS3AHugeFiles.java} (93%) create mode 100644 hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/STestS3AHugeFilesDiskBlocks.java create mode 100644 hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/STestS3AHugeFilesFastUpload.java create mode 100644 hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/STestS3AHugeFilesMemoryBlocks.java diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java index 4105983cfce8b..d23c48982282d 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java @@ -88,6 +88,7 @@ class S3ABlockOutputStream extends OutputStream { private MultiPartUpload multiPartUpload; private volatile boolean closed; private S3ADataBlocks.DataBlock currentBlock; + private long blockCount = 0; /** * S3A output stream which uploads blocks as soon as there is enough @@ -131,6 +132,7 @@ public S3ABlockOutputStream(S3AFileSystem fs, public void maybeCreateDestStream() throws IOException { if (currentBlock == null) { + blockCount++; currentBlock = blockFactory.create(this.blockSize); } } @@ -145,6 +147,20 @@ void checkOpen() throws IOException { } } + /** + * 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(); + if (currentBlock != null) { + currentBlock.flush(); + } + } + /** * Writes a byte to the destination. If this causes the buffer to reach * its limit, the actual upload is submitted to the threadpool. @@ -189,7 +205,6 @@ public synchronized void write(byte[] source, int offset, int len) } else { if (currentBlock.remainingCapacity() == 0) { // the whole buffer is done, trigger an upload - LOG.debug("Current block is full"); uploadCurrentBlock(); } } @@ -201,6 +216,7 @@ public synchronized void write(byte[] source, int offset, int len) * or initializing the upload. */ private synchronized void uploadCurrentBlock() throws IOException { + LOG.debug("Writing block # {}", blockCount); if (multiPartUpload == null) { multiPartUpload = initiateMultiPartUpload(); } @@ -221,25 +237,27 @@ public synchronized void close() throws IOException { if (closed) { return; } - LOG.debug("Closing {}, data to upload = {}", this, - currentBlock == null ? 0 : currentBlock.dataSize() ); closed = true; - if (currentBlock == null) { - return; - } + LOG.debug("{}: Closing block #{}: {}, data to upload = {}", + this, + blockCount, + currentBlock, + currentBlock == null ? 0 : currentBlock.dataSize() ); try { - if (multiPartUpload == null) { + if (multiPartUpload == null && currentBlock != null) { // no uploads of data have taken place, put the single block up. putObject(); } else { // there has already been at least one block scheduled for upload; // put up the current then wait - if (currentBlock.hasData()) { + if (currentBlock != null && currentBlock.hasData()) { //send last part multiPartUpload.uploadBlockAsync(currentBlock); } - final List partETags = multiPartUpload - .waitForAllPartUploads(); + // wait for the partial uploads to finish + final List partETags = + multiPartUpload.waitForAllPartUploads(); + // then complete the operation multiPartUpload.complete(partETags); } // This will delete unnecessary fake parent directories @@ -378,6 +396,7 @@ public PartETag call() throws Exception { } private List waitForAllPartUploads() throws IOException { + LOG.debug("Waiting for {} uploads to complete", partETagsFutures.size()); try { return Futures.allAsList(partETagsFutures).get(); } catch (InterruptedException ie) { @@ -387,6 +406,8 @@ private List waitForAllPartUploads() throws IOException { } catch (ExecutionException ee) { //there is no way of recovering so abort //cancel all partUploads + LOG.debug("While waiting for upload completion", ee); + LOG.debug("Cancelling futures"); for (ListenableFuture future : partETagsFutures) { future.cancel(true); } @@ -410,8 +431,9 @@ private CompleteMultipartUploadResult complete(List partETags) AmazonClientException lastException; do { try { - LOG.debug("Completing multi-part upload for key '{}', id '{}'" , - key, uploadId); + LOG.debug("Completing multi-part upload for key '{}'," + + " id '{}' with {} partitions " , + key, uploadId, partETags.size()); return getClient().completeMultipartUpload( new CompleteMultipartUploadRequest(bucket, key, diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ADataBlocks.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ADataBlocks.java index 7e5d79320cb4d..2d1532ce90e78 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ADataBlocks.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ADataBlocks.java @@ -22,6 +22,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.io.BufferedInputStream; import java.io.BufferedOutputStream; import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; @@ -279,7 +280,7 @@ static class FileBlock extends DataBlock { protected int bytesWritten; private BufferedOutputStream out; - private FileInputStream uploadStream; + private InputStream uploadStream; public FileBlock(File bufferFile, int limit) throws FileNotFoundException { @@ -322,7 +323,7 @@ InputStream openForUpload() throws IOException { out.close(); out = null; } - uploadStream = new FileInputStream(bufferFile); + uploadStream = new BufferedInputStream(new FileInputStream(bufferFile)); return new FileDeletingInputStream(uploadStream); } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ABlockOutputDisk.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ABlockOutputDisk.java new file mode 100644 index 0000000000000..3f0dc8fad0a7f --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ABlockOutputDisk.java @@ -0,0 +1,27 @@ +/* + * 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; + +public class ITestS3ABlockOutputDisk extends ITestS3ABlockOutputMemory { + + protected String getBlockOutputBufferName() { + return Constants.BLOCK_OUTPUT_BUFFER_DISK; + } + +} diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ABlockOutputMemory.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ABlockOutputMemory.java index c105f506440aa..ac73c9365b505 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ABlockOutputMemory.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ABlockOutputMemory.java @@ -19,19 +19,16 @@ package org.apache.hadoop.fs.s3a; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.contract.ContractTestUtils; -import org.junit.After; -import org.junit.Before; import org.junit.Rule; import org.junit.Test; import org.junit.rules.Timeout; -import static org.apache.hadoop.fs.s3a.Constants.*; - import java.io.IOException; +import static org.apache.hadoop.fs.s3a.Constants.*; + /** * Tests regular and multi-part upload functionality for * {@link S3ABlockOutputStream} with the block buffered in memory. @@ -58,6 +55,11 @@ protected String getBlockOutputBufferName() { return BLOCK_OUTPUT_BUFFER_ARRAY; } + @Test + public void testZeroByteUpload() throws IOException { + verifyUpload("0", 0); + } + @Test public void testRegularUpload() throws IOException { verifyUpload("regular", 1024); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/STestS3AHugeFileCreate.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/AbstractSTestS3AHugeFiles.java similarity index 93% rename from hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/STestS3AHugeFileCreate.java rename to hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/AbstractSTestS3AHugeFiles.java index 6c61a7b9e35c0..0f8d74220365a 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/STestS3AHugeFileCreate.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/AbstractSTestS3AHugeFiles.java @@ -26,11 +26,8 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.StorageStatistics; import org.apache.hadoop.fs.contract.ContractTestUtils; -import org.apache.hadoop.fs.s3a.Constants; import org.apache.hadoop.fs.s3a.S3AFileStatus; -import org.apache.hadoop.fs.s3a.S3AInstrumentation; import org.apache.hadoop.fs.s3a.Statistic; -import org.apache.hadoop.metrics2.lib.MutableGaugeLong; import org.apache.hadoop.util.Progressable; import org.junit.Assume; import org.junit.FixMethodOrder; @@ -58,9 +55,9 @@ * check at the start, in case an individual test is executed. */ @FixMethodOrder(MethodSorters.NAME_ASCENDING) -public class STestS3AHugeFileCreate extends S3AScaleTestBase { +public abstract class AbstractSTestS3AHugeFiles extends S3AScaleTestBase { private static final Logger LOG = LoggerFactory.getLogger( - STestS3AHugeFileCreate.class); + AbstractSTestS3AHugeFiles.class); private Path scaleTestDir; private Path hugefile; private Path hugefileRenamed; @@ -84,10 +81,11 @@ public void tearDown() throws Exception { @Override protected Configuration createConfiguration() { Configuration configuration = super.createConfiguration(); - configuration.setBoolean(FAST_UPLOAD, true); - configuration.setLong(MIN_MULTIPART_THRESHOLD, 5 * _1MB); configuration.setLong(SOCKET_SEND_BUFFER, BLOCKSIZE); configuration.setLong(SOCKET_RECV_BUFFER, BLOCKSIZE); + configuration.setLong(MIN_MULTIPART_THRESHOLD, 5 * _1MB); + configuration.setLong(MIN_MULTIPART_THRESHOLD, MULTIPART_MIN_SIZE); + configuration.setInt(MULTIPART_SIZE, MULTIPART_MIN_SIZE); configuration.set(USER_AGENT_PREFIX, "STestS3AHugeFileCreate"); return configuration; } @@ -98,17 +96,6 @@ public void test_010_CreateHugeFile() throws IOException { long filesize = _1MB * mb; describe("Creating file %s of size %d MB", hugefile, mb); - try { - S3AFileStatus status = fs.getFileStatus(hugefile); - long actualSize = status.getLen(); - if (actualSize == filesize) { - String message = "File of size " + mb + " MB exists: " + status; - LOG.info(message); - throw new AssumptionViolatedException(message); - } - } catch (FileNotFoundException e) { - // the file doesn't exist and so must be created. - } byte[] data = new byte[BLOCKSIZE]; for (int i = 0; i < BLOCKSIZE; i++) { data[i] = (byte)(i % 256); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/STestS3AHugeFilesDiskBlocks.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/STestS3AHugeFilesDiskBlocks.java new file mode 100644 index 0000000000000..9731102e7e95b --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/STestS3AHugeFilesDiskBlocks.java @@ -0,0 +1,40 @@ +/* + * 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.scale; + +import org.apache.hadoop.conf.Configuration; + +import static org.apache.hadoop.fs.s3a.Constants.BLOCK_OUTPUT; +import static org.apache.hadoop.fs.s3a.Constants.BLOCK_OUTPUT_BUFFER; +import static org.apache.hadoop.fs.s3a.Constants.BLOCK_OUTPUT_BUFFER_DISK; +import static org.apache.hadoop.fs.s3a.Constants.FAST_UPLOAD; +import static org.apache.hadoop.fs.s3a.Constants.MIN_MULTIPART_THRESHOLD; +import static org.apache.hadoop.fs.s3a.Constants.MULTIPART_MIN_SIZE; +import static org.apache.hadoop.fs.s3a.Constants.MULTIPART_SIZE; + +public class STestS3AHugeFilesDiskBlocks extends AbstractSTestS3AHugeFiles { + + @Override + protected Configuration createConfiguration() { + final Configuration configuration = super.createConfiguration(); + configuration.setBoolean(BLOCK_OUTPUT, true); + configuration.set(BLOCK_OUTPUT_BUFFER, BLOCK_OUTPUT_BUFFER_DISK); + return configuration; + } +} diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/STestS3AHugeFilesFastUpload.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/STestS3AHugeFilesFastUpload.java new file mode 100644 index 0000000000000..7e6da5791ffb2 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/STestS3AHugeFilesFastUpload.java @@ -0,0 +1,34 @@ +/* + * 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.scale; + +import org.apache.hadoop.conf.Configuration; + +import static org.apache.hadoop.fs.s3a.Constants.FAST_UPLOAD; +import static org.apache.hadoop.fs.s3a.Constants.MIN_MULTIPART_THRESHOLD; + +public class STestS3AHugeFilesFastUpload extends AbstractSTestS3AHugeFiles { + + @Override + protected Configuration createConfiguration() { + final Configuration configuration = super.createConfiguration(); + configuration.setBoolean(FAST_UPLOAD, true); + return configuration; + } +} diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/STestS3AHugeFilesMemoryBlocks.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/STestS3AHugeFilesMemoryBlocks.java new file mode 100644 index 0000000000000..c3a930fbd6e7f --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/STestS3AHugeFilesMemoryBlocks.java @@ -0,0 +1,37 @@ +/* + * 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.scale; + +import org.apache.hadoop.conf.Configuration; + +import static org.apache.hadoop.fs.s3a.Constants.BLOCK_OUTPUT; +import static org.apache.hadoop.fs.s3a.Constants.BLOCK_OUTPUT_BUFFER; +import static org.apache.hadoop.fs.s3a.Constants.BLOCK_OUTPUT_BUFFER_ARRAY; +import static org.apache.hadoop.fs.s3a.Constants.BLOCK_OUTPUT_BUFFER_DISK; + +public class STestS3AHugeFilesMemoryBlocks extends AbstractSTestS3AHugeFiles { + + @Override + protected Configuration createConfiguration() { + final Configuration configuration = super.createConfiguration(); + configuration.setBoolean(BLOCK_OUTPUT, true); + configuration.set(BLOCK_OUTPUT_BUFFER, BLOCK_OUTPUT_BUFFER_ARRAY); + return configuration; + } +} From 1e8c6de4120cab6ae4a0dcdc18b97afad4bf1223 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Wed, 7 Sep 2016 15:12:11 +0100 Subject: [PATCH 15/20] HADOOP-13560 ongoing work on disk uploads at 2+ GB scale. --- .../hadoop/fs/s3a/S3ABlockOutputStream.java | 10 ++- .../apache/hadoop/fs/s3a/S3ADataBlocks.java | 73 +++++++++++-------- .../s3a/scale/AbstractSTestS3AHugeFiles.java | 60 ++++++++++++++- .../fs/s3a/scale/STestS3AHugeDistCP.java | 2 +- 4 files changed, 104 insertions(+), 41 deletions(-) diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java index d23c48982282d..ecba94367fcd3 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java @@ -130,7 +130,7 @@ public S3ABlockOutputStream(S3AFileSystem fs, maybeCreateDestStream(); } - public void maybeCreateDestStream() throws IOException { + private synchronized void maybeCreateDestStream() throws IOException { if (currentBlock == null) { blockCount++; currentBlock = blockFactory.create(this.blockSize); @@ -244,9 +244,11 @@ public synchronized void close() throws IOException { currentBlock, currentBlock == null ? 0 : currentBlock.dataSize() ); try { - if (multiPartUpload == null && currentBlock != null) { - // no uploads of data have taken place, put the single block up. - putObject(); + if (multiPartUpload == null) { + if (currentBlock != null){ + // no uploads of data have taken place, put the single block up. + putObject(); + } } else { // there has already been at least one block scheduled for upload; // put up the current then wait diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ADataBlocks.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ADataBlocks.java index 2d1532ce90e78..5ff32c80cfb5d 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ADataBlocks.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ADataBlocks.java @@ -22,7 +22,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.BufferedInputStream; import java.io.BufferedOutputStream; import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; @@ -42,7 +41,6 @@ */ class S3ADataBlocks { static final Logger LOG = LoggerFactory.getLogger(S3ADataBlocks.class); - ; static abstract class AbstractBlockFactory implements Closeable { @@ -50,10 +48,10 @@ static abstract class AbstractBlockFactory implements Closeable { /** * Bind to the factory owner. - * @param owner owner factory + * @param fs owner filesystem */ - void init(S3AFileSystem owner) { - this.owner = owner; + void init(S3AFileSystem fs) { + this.owner = fs; } /** @@ -76,8 +74,8 @@ public void close() throws IOException { static class MemoryBlockFactory extends AbstractBlockFactory { @Override - void init(S3AFileSystem owner) { - super.init(owner); + void init(S3AFileSystem fs) { + super.init(fs); } @Override @@ -94,8 +92,8 @@ DataBlock create(int limit) throws IOException { static class DiskBlockFactory extends AbstractBlockFactory { @Override - void init(S3AFileSystem owner) { - super.init(owner); + void init(S3AFileSystem fs) { + super.init(fs); } @Override @@ -137,8 +135,8 @@ DestState getState() { abstract boolean hasCapacity(long bytes); /** - * Is there data in the block? - * @return + * Is there data in the block. + * @return true if there is */ boolean hasData() { return dataSize() > 0; @@ -149,11 +147,11 @@ boolean hasData() { /** * Write a series of bytes from the buffer, from the offset. * Returns the number of bytes written: - * @param b - * @param off - * @param len + * @param b buffer + * @param off offset + * @param len length of write * @return number of bytes written - * @throws IOException + * @throws IOException trouble */ int write(byte b[], int off, int len) throws IOException { verifyState(Writing); @@ -167,7 +165,7 @@ void flush() throws IOException { /** * Switch to the upload state and return a stream for uploading. * @return the stream - * @throws IOException + * @throws IOException trouble */ InputStream openForUpload() throws IOException { enterState(Writing, Upload); @@ -178,7 +176,6 @@ InputStream openForUpload() throws IOException { * Enter the closed state. * @return true if the class was in any other state, implying that * the subclass should do its close operations - * @throws IOException */ protected synchronized boolean enterClosedState() { if (!state.equals(Closed)) { @@ -323,26 +320,31 @@ InputStream openForUpload() throws IOException { out.close(); out = null; } - uploadStream = new BufferedInputStream(new FileInputStream(bufferFile)); + uploadStream = new FileInputStream(bufferFile); return new FileDeletingInputStream(uploadStream); } @Override public synchronized void close() throws IOException { + final DestState state = getState(); + LOG.debug("Closing {}", this); enterClosedState(); - LOG.debug("Closed {}", this); - if (bufferFile.exists()) { - LOG.warn("Buffer file {} exists —close upload stream", bufferFile); + final boolean bufferExists = bufferFile.exists(); + switch (state) { + case Writing: + if (bufferExists) { + // file was not uploaded + LOG.debug("Deleting buffer file as upload did not start"); + bufferFile.delete(); + } + break; + case Upload: + LOG.debug("Buffer file {} exists —close upload stream", bufferFile); + break; + + case Closed: + // no-op } -/* - IOUtils.closeStream(out); - out = null; - if (uploadStream != null) { - IOUtils.closeStream(uploadStream); - } else { - destFile.delete(); - } -*/ } @Override @@ -363,9 +365,12 @@ public String toString() { return sb.toString(); } - class FileDeletingInputStream extends ForwardingInputStream { + /** + * An input stream which deletes the buffer file when closed. + */ + private class FileDeletingInputStream extends ForwardingInputStream { - public FileDeletingInputStream(InputStream source) { + FileDeletingInputStream(InputStream source) { super(source); } @@ -381,6 +386,10 @@ public void close() throws IOException { * Stream which forwards everything to its inner class. * For ease of subclassing. */ + @SuppressWarnings({ + "NullableProblems", + "NonSynchronizedMethodOverridesSynchronizedMethod" + }) static class ForwardingInputStream extends InputStream { protected final InputStream source; diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/AbstractSTestS3AHugeFiles.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/AbstractSTestS3AHugeFiles.java index 0f8d74220365a..750ad840f15b9 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/AbstractSTestS3AHugeFiles.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/AbstractSTestS3AHugeFiles.java @@ -200,6 +200,59 @@ void assumeHugeFileExists() throws IOException { Assume.assumeTrue("No file " + hugefile, fs.exists(hugefile)); } + private void logFSState() { + LOG.info("File System state after operation:\n{}", fs); + } + + @Test + public void test_040_PositionedReadHugeFile() throws Throwable { + assumeHugeFileExists(); + final String encryption = getConf().getTrimmed( + SERVER_SIDE_ENCRYPTION_ALGORITHM); + boolean encrypted = encryption != null; + if (encrypted) { + LOG.info("File is encrypted with algorithm {}", encryption); + } + String filetype = encrypted ? "encrypted file" : "file"; + describe("Positioned reads of %s %s", filetype, hugefile); + S3AFileStatus status = fs.getFileStatus(hugefile); + long filesize = status.getLen(); + int ops = 0; + final int bufferSize = 8192; + byte[] buffer = new byte[bufferSize]; + long eof = filesize - 1; + + ContractTestUtils.NanoTimer timer = new ContractTestUtils.NanoTimer(); + ContractTestUtils.NanoTimer readAtByte0, readAtByte0Again, readAtEOF; + try (FSDataInputStream in = fs.open(hugefile, BLOCKSIZE)) { + readAtByte0 = new ContractTestUtils.NanoTimer(); + in.readFully(0, buffer); + readAtByte0.end("Time to read data at start of file"); + ops++; + + readAtEOF = new ContractTestUtils.NanoTimer(); + in.readFully(eof - bufferSize, buffer); + readAtEOF.end("Time to read data at end of file"); + ops++; + + readAtByte0Again = new ContractTestUtils.NanoTimer(); + in.readFully(0, buffer); + readAtByte0.end("Time to read data at start of file again"); + ops++; + LOG.info("Final stream state: {}", in); + } + long mb = Math.max(filesize / _1MB, 1); + + logFSState(); + timer.end("Time to performed positioned reads of %s of %d MB ", + filetype, mb); + LOG.info("Time per positioned read = {} nS", + toHuman(timer.nanosPerOperation(ops))); + final long difference = readAtEOF.duration() - readAtByte0Again.duration(); + LOG.info("Difference between read at start & end of {} is {} nS", + filetype, toHuman(difference)); + } + @Test public void test_050_readHugeFile() throws Throwable { assumeHugeFileExists(); @@ -214,18 +267,17 @@ public void test_050_readHugeFile() throws Throwable { for (long block = 0; block < blocks; block++) { in.readFully(data); } + LOG.info("Final stream state: {}", in); } long mb = Math.max(filesize / _1MB, 1); timer.end("Time to read file of %d MB ", mb); - LOG.info("Time per MB to read = {} nS", toHuman(timer.duration() / mb)); + LOG.info("Time per MB to read = {} nS", + toHuman(timer.nanosPerOperation(mb))); LOG.info("Effective Bandwidth: {} MB/s", timer.bandwidth(filesize)); logFSState(); } - private void logFSState() { - LOG.info("File System state after operation:\n{}", fs); - } @Test public void test_100_renameHugeFile() throws Throwable { diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/STestS3AHugeDistCP.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/STestS3AHugeDistCP.java index d6598d6d9952e..75c1794cabf89 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/STestS3AHugeDistCP.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/STestS3AHugeDistCP.java @@ -22,7 +22,7 @@ import org.slf4j.LoggerFactory; /** - * Test very large scale distCP upload using the huge file options + * Test very large scale distCP upload using the huge file options. */ public class STestS3AHugeDistCP extends S3AScaleTestBase { private static final Logger LOG = LoggerFactory.getLogger( From 272dd5a8d59134cb10b5312a381b80cc25caedde Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Wed, 7 Sep 2016 16:12:16 +0100 Subject: [PATCH 16/20] HADOOP-13560 complete merge with branch-2. Milestone: 1GB file round trip @ 1.57 MB/s --- .../hadoop/fs/s3a/S3ABlockOutputStream.java | 16 +++++++--------- .../apache/hadoop/fs/s3a/S3AFileSystem.java | 13 ++++--------- .../org/apache/hadoop/fs/s3a/S3AUtils.java | 18 +++++++++++++----- 3 files changed, 24 insertions(+), 23 deletions(-) diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java index ecba94367fcd3..97d06cb67cd80 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java @@ -21,7 +21,7 @@ import com.amazonaws.AmazonClientException; import com.amazonaws.event.ProgressEvent; import com.amazonaws.event.ProgressListener; -import com.amazonaws.services.s3.AmazonS3Client; +import com.amazonaws.services.s3.AmazonS3; import com.amazonaws.services.s3.model.AbortMultipartUploadRequest; import com.amazonaws.services.s3.model.CannedAccessControlList; import com.amazonaws.services.s3.model.CompleteMultipartUploadRequest; @@ -55,10 +55,8 @@ import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; -import static org.apache.hadoop.fs.s3a.S3AUtils.extractException; -import static org.apache.hadoop.fs.s3a.S3AUtils.translateException; -import static org.apache.hadoop.fs.s3a.Statistic.IGNORED_ERRORS; -import static org.apache.hadoop.fs.s3a.Statistic.OBJECT_MULTIPART_UPLOAD_ABORTED; +import static org.apache.hadoop.fs.s3a.S3AUtils.*; +import static org.apache.hadoop.fs.s3a.Statistic.*; /** * Upload files/parts directly via different buffering mechanisms: @@ -290,7 +288,7 @@ private MultiPartUpload initiateMultiPartUpload() throws IOException { initiateMPURequest.setCannedACL(cannedACL); try { return new MultiPartUpload( - getClient().initiateMultipartUpload(initiateMPURequest).getUploadId()); + getS3Client().initiateMultipartUpload(initiateMPURequest).getUploadId()); } catch (AmazonClientException ace) { throw translateException("initiate MultiPartUpload", key, ace); } @@ -334,7 +332,7 @@ public PutObjectResult call() throws Exception { } } - private AmazonS3Client getClient() { + private AmazonS3 getS3Client() { return fs.getAmazonS3Client(); } @@ -436,7 +434,7 @@ private CompleteMultipartUploadResult complete(List partETags) LOG.debug("Completing multi-part upload for key '{}'," + " id '{}' with {} partitions " , key, uploadId, partETags.size()); - return getClient().completeMultipartUpload( + return getS3Client().completeMultipartUpload( new CompleteMultipartUploadRequest(bucket, key, uploadId, @@ -462,7 +460,7 @@ public void abort() { try { LOG.debug("Aborting multi-part upload for key '{}', id '{}'", key, uploadId); - getClient().abortMultipartUpload(new AbortMultipartUploadRequest(bucket, + getS3Client().abortMultipartUpload(new AbortMultipartUploadRequest(bucket, key, uploadId)); return; } catch (AmazonClientException e) { diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java index b35bd38c159f0..1df67dec4b58b 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java @@ -161,8 +161,8 @@ public void initialize(URI name, Configuration conf) throws IOException { maxKeys = intOption(conf, MAX_PAGING_KEYS, DEFAULT_MAX_PAGING_KEYS, 1); listing = new Listing(this); - partSize = getSizeProperty(conf, MULTIPART_SIZE, DEFAULT_MULTIPART_SIZE); - multiPartThreshold = getSizeProperty(conf, MIN_MULTIPART_THRESHOLD, + partSize = getMultipartSizeProperty(conf, MULTIPART_SIZE, DEFAULT_MULTIPART_SIZE); + multiPartThreshold = getMultipartSizeProperty(conf, MIN_MULTIPART_THRESHOLD, DEFAULT_MIN_MULTIPART_THRESHOLD); //check but do not store the block size @@ -217,17 +217,12 @@ public StorageStatistics provide() { conf.getTrimmed(SERVER_SIDE_ENCRYPTION_ALGORITHM); inputPolicy = S3AInputPolicy.getPolicy( conf.getTrimmed(INPUT_FADVISE, INPUT_FADV_NORMAL)); - fastUploadEnabled = getConf().getBoolean(FAST_UPLOAD, - DEFAULT_FAST_UPLOAD); - String itemName = conf.get(BUFFER_DIR) != null - ? BUFFER_DIR : "hadoop.tmp.dir"; - directoryAllocator = new LocalDirAllocator(itemName); blockUploadEnabled = conf.getBoolean(BLOCK_OUTPUT, false); if (blockUploadEnabled) { blockOutputBuffer = conf.getTrimmed(BLOCK_OUTPUT_BUFFER, DEFAULT_BLOCK_OUTPUT_BUFFER); - partSize = ensureIntVal(MULTIPART_SIZE, partSize); + partSize = ensureOutputParameterInRange(MULTIPART_SIZE, partSize); blockFactory = S3ADataBlocks.createFactory(blockOutputBuffer); blockFactory.init(this); LOG.debug("Uploading data via Block Upload, buffer = {}", @@ -236,7 +231,7 @@ public StorageStatistics provide() { fastUploadEnabled = getConf().getBoolean(FAST_UPLOAD, DEFAULT_FAST_UPLOAD); if (fastUploadEnabled) { - partSize = ensureIntVal(MULTIPART_SIZE, partSize); + partSize = ensureOutputParameterInRange(MULTIPART_SIZE, partSize); } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java index bbbed6a389aa0..06a18c3b40332 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java @@ -48,8 +48,6 @@ import static org.apache.hadoop.fs.s3a.Constants.ACCESS_KEY; import static org.apache.hadoop.fs.s3a.Constants.AWS_CREDENTIALS_PROVIDER; -import static org.apache.hadoop.fs.s3a.Constants.DEFAULT_MIN_MULTIPART_THRESHOLD; -import static org.apache.hadoop.fs.s3a.Constants.MIN_MULTIPART_THRESHOLD; import static org.apache.hadoop.fs.s3a.Constants.MULTIPART_MIN_SIZE; import static org.apache.hadoop.fs.s3a.Constants.SECRET_KEY; @@ -459,11 +457,21 @@ static long longOption(Configuration conf, return v; } - public static long getSizeProperty(Configuration conf, + /** + * Get a size property from the configuration: this property must + * be at least equal to {@link Constants#MULTIPART_MIN_SIZE}. + * If it is too small, it is rounded up to that minimum, and a warning + * printed. + * @param conf configuration + * @param property property name + * @param defVal default value + * @return the value, guaranteed to be above the minimum size + */ + public static long getMultipartSizeProperty(Configuration conf, String property, long defVal) { long partSize = conf.getLong(property, defVal); if (partSize < MULTIPART_MIN_SIZE) { - LOG.error(property + " must be at least 5 MB"); + LOG.warn(property + " must be at least 5 MB"); partSize = MULTIPART_MIN_SIZE; } return partSize; @@ -476,7 +484,7 @@ public static long getSizeProperty(Configuration conf, * @return the size, guaranteed to be less than or equal to the max * value of an integer. */ - public static int ensureIntVal(String name, long size) { + public static int ensureOutputParameterInRange(String name, long size) { if (size > Integer.MAX_VALUE) { LOG.warn("s3a: " + name + " capped to ~2.14GB" + " (maximum allowed size with current output mechanism)"); From 22b8de79f3e76d85d41912eb938f34601d55e085 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Fri, 9 Sep 2016 11:17:39 +0100 Subject: [PATCH 17/20] HADOOP-13560 block output stream collects statistics --- .../hadoop/fs/s3a/S3ABlockOutputStream.java | 110 +++++++++++- .../apache/hadoop/fs/s3a/S3ADataBlocks.java | 8 + .../apache/hadoop/fs/s3a/S3AFileSystem.java | 3 +- .../hadoop/fs/s3a/S3AInstrumentation.java | 169 +++++++++++++++++- .../org/apache/hadoop/fs/s3a/Statistic.java | 28 ++- .../s3a/scale/AbstractSTestS3AHugeFiles.java | 9 +- 6 files changed, 308 insertions(+), 19 deletions(-) diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java index 97d06cb67cd80..9230652846f94 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java @@ -20,6 +20,7 @@ import com.amazonaws.AmazonClientException; import com.amazonaws.event.ProgressEvent; +import com.amazonaws.event.ProgressEventType; import com.amazonaws.event.ProgressListener; import com.amazonaws.services.s3.AmazonS3; import com.amazonaws.services.s3.model.AbortMultipartUploadRequest; @@ -87,6 +88,8 @@ class S3ABlockOutputStream extends OutputStream { private volatile boolean closed; private S3ADataBlocks.DataBlock currentBlock; private long blockCount = 0; + private final S3AInstrumentation.OutputStreamStatistics statistics; + /** * S3A output stream which uploads blocks as soon as there is enough @@ -99,13 +102,15 @@ class S3ABlockOutputStream extends OutputStream { * 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 * @throws IOException on any problem */ - public S3ABlockOutputStream(S3AFileSystem fs, + S3ABlockOutputStream(S3AFileSystem fs, String key, Progressable progress, long blockSize, - S3ADataBlocks.AbstractBlockFactory blockFactory) + S3ADataBlocks.AbstractBlockFactory blockFactory, + S3AInstrumentation.OutputStreamStatistics statistics) throws IOException { this.fs = fs; this.bucket = fs.getBucket(); @@ -113,6 +118,7 @@ public S3ABlockOutputStream(S3AFileSystem fs, this.cannedACL = fs.getCannedACL(); this.blockFactory = blockFactory; this.blockSize = (int) blockSize; + this.statistics = statistics; Preconditions.checkArgument(blockSize >= Constants.MULTIPART_MIN_SIZE, "Block size is too small: %d", blockSize); this.executorService = MoreExecutors.listeningDecorator( @@ -128,6 +134,10 @@ public S3ABlockOutputStream(S3AFileSystem fs, maybeCreateDestStream(); } + /** + * Demand create a destination stream. + * @throws IOException on any failure to create + */ private synchronized void maybeCreateDestStream() throws IOException { if (currentBlock == null) { blockCount++; @@ -267,6 +277,8 @@ public synchronized void close() throws IOException { LOG.debug("Closing block and factory"); IOUtils.closeStream(currentBlock); IOUtils.closeStream(blockFactory); + LOG.debug("Statistics: {}", statistics); + IOUtils.closeStream(statistics); currentBlock = null; } } @@ -310,7 +322,12 @@ private void putObject() throws IOException { fs.newPutObjectRequest(key, om, block.openForUpload()); - putObjectRequest.setGeneralProgressListener(progressListener); + long transferQueueTime = now(); + BlockUploadProgress callback = + new BlockUploadProgress( + block, progressListener, transferQueueTime); + putObjectRequest.setGeneralProgressListener(callback); + statistics.blockUploadQueued(block.dataSize()); ListenableFuture putObjectResult = executorService.submit(new Callable() { @Override @@ -343,11 +360,21 @@ public String toString() { sb.append("key='").append(key).append('\''); sb.append(", bucket='").append(bucket).append('\''); sb.append(", blockSize=").append(blockSize); - sb.append(", dest=").append(currentBlock); + if (currentBlock != null) { + sb.append(", currentBlock=").append(currentBlock); + } sb.append('}'); return sb.toString(); } + private void incrementWriteOperations() { + fs.incrementWriteOperations(); + } + + private long now() { + return System.currentTimeMillis(); + } + /** * Multiple partition upload. */ @@ -371,13 +398,18 @@ private void uploadBlockAsync(final S3ADataBlocks.DataBlock block) throws IOException { LOG.debug("Queueing upload of {}", block); final int size = block.dataSize(); - final InputStream inputStream = block.openForUpload(); + final InputStream uploadStream = block.openForUpload(); final int currentPartNumber = partETagsFutures.size() + 1; final UploadPartRequest request = new UploadPartRequest().withBucketName(bucket).withKey(key) - .withUploadId(uploadId).withInputStream(inputStream) + .withUploadId(uploadId).withInputStream(uploadStream) .withPartNumber(currentPartNumber).withPartSize(size); - request.setGeneralProgressListener(progressListener); + long transferQueueTime = now(); + BlockUploadProgress callback = + new BlockUploadProgress( + block, progressListener, transferQueueTime); + request.setGeneralProgressListener(callback); + statistics.blockUploadQueued(block.dataSize()); ListenableFuture partETagFuture = executorService.submit(new Callable() { @Override @@ -387,7 +419,10 @@ public PartETag call() throws Exception { uploadId); // do the upload PartETag partETag = fs.uploadPart(request).getPartETag(); - // close the block, triggering its cleanup + LOG.debug("Completed upload of {}", block); + LOG.debug("Stream statistics of {}", statistics); + + // close the block block.close(); return partETag; } @@ -395,7 +430,7 @@ public PartETag call() throws Exception { partETagsFutures.add(partETagFuture); } - private List waitForAllPartUploads() throws IOException { + private List waitForAllPartUploads() throws IOException { LOG.debug("Waiting for {} uploads to complete", partETagsFutures.size()); try { return Futures.allAsList(partETagsFutures).get(); @@ -441,6 +476,7 @@ private CompleteMultipartUploadResult complete(List partETags) partETags)); } catch (AmazonClientException e) { lastException = e; + statistics.exceptionInMultipartComplete(); } } while (shouldRetry(lastException, retryCount++)); // this point is only reached if the operation failed more than @@ -465,6 +501,7 @@ public void abort() { return; } catch (AmazonClientException e) { lastException = e; + statistics.exceptionInMultipartAbort(); } } while (shouldRetry(lastException, retryCount++)); // this point is only reached if the operation failed more than @@ -496,7 +533,62 @@ private boolean shouldRetry(AmazonClientException e, int retryCount) { } } + } + + /** + * The upload progress listener registered for events. + * It updates statistics and handles the end of the upload + */ + private class BlockUploadProgress implements ProgressListener { + private final S3ADataBlocks.DataBlock block; + private final ProgressListener next; + private final long transferQueueTime; + private long transferStartTime; + + private BlockUploadProgress(S3ADataBlocks.DataBlock block, + ProgressListener next, + long transferQueueTime) { + this.block = block; + this.transferQueueTime = transferQueueTime; + this.next = next; + } + @Override + public void progressChanged(ProgressEvent progressEvent) { + ProgressEventType eventType = progressEvent.getEventType(); + long bytesTransferred = progressEvent.getBytesTransferred(); + + int blockSize = block.dataSize(); + switch (eventType) { + + case REQUEST_BYTE_TRANSFER_EVENT: + // bytes uploaded + statistics.bytesTransferred(bytesTransferred); + break; + + case TRANSFER_PART_STARTED_EVENT: + transferStartTime = now(); + statistics.blockUploadStarted(transferStartTime - transferQueueTime, + blockSize); + incrementWriteOperations(); + break; + + case TRANSFER_PART_COMPLETED_EVENT: + statistics.blockUploadCompleted(now() - transferStartTime, blockSize); + break; + + case TRANSFER_PART_FAILED_EVENT: + statistics.blockUploadFailed(now() - transferStartTime, blockSize); + break; + + default: + // nothing + } + + if (next != null) { + next.progressChanged(progressEvent); + } + } } private static class ProgressableListener implements ProgressListener { diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ADataBlocks.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ADataBlocks.java index 5ff32c80cfb5d..fa97a873939c7 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ADataBlocks.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ADataBlocks.java @@ -172,6 +172,14 @@ InputStream openForUpload() throws IOException { return null; } + /** + * actions to take on block upload completion. + * @throws IOException Any failure + */ + void blockUploadCompleted() throws IOException { + + } + /** * Enter the closed state. * @return true if the class was in any other state, implying that diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java index 1df67dec4b58b..f1d5a0d00e710 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java @@ -547,7 +547,8 @@ public FSDataOutputStream create(Path f, FsPermission permission, key, progress, partSize, - blockFactory), + blockFactory, + instrumentation.newOutputStreamStatistics()), null); } else if (fastUploadEnabled) { output = new FSDataOutputStream( diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java index 2f14ffc5c093b..15ce2bc7219b7 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java @@ -18,7 +18,6 @@ package org.apache.hadoop.fs.s3a; -import com.google.common.base.Preconditions; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.metrics2.MetricStringBuilder; @@ -31,10 +30,12 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.io.Closeable; import java.net.URI; import java.util.HashMap; import java.util.Map; import java.util.UUID; +import java.util.concurrent.atomic.AtomicLong; import static org.apache.hadoop.fs.s3a.Statistic.*; @@ -105,12 +106,20 @@ public class S3AInstrumentation { OBJECT_MULTIPART_UPLOAD_ABORTED, OBJECT_PUT_BYTES, OBJECT_PUT_REQUESTS, - OBJECT_PUT_REQUESTS_COMPLETED + OBJECT_PUT_REQUESTS_COMPLETED, + STREAM_WRITE_FAILURES, + STREAM_WRITE_BLOCK_UPLOADS, + STREAM_WRITE_BLOCK_UPLOADS_COMMITTED, + STREAM_WRITE_BLOCK_UPLOADS_ABORTED, + STREAM_WRITE_TOTAL_TIME, + STREAM_WRITE_TOTAL_DATA, }; private static final Statistic[] GAUGES_TO_CREATE = { OBJECT_PUT_REQUESTS_ACTIVE, - OBJECT_PUT_BYTES_PENDING + OBJECT_PUT_BYTES_PENDING, + STREAM_WRITE_BLOCK_UPLOADS_DATA_PENDING, + STREAM_WRITE_BANDWIDTH, }; public S3AInstrumentation(URI name) { @@ -367,6 +376,15 @@ public void incrementCounter(Statistic op, long count) { counter.incr(count); } } + /** + * Increment a specific counter. + * No-op if not defined. + * @param op operation + * @param count atomic long containing value + */ + public void incrementCounter(Statistic op, AtomicLong count) { + incrementCounter(op, count.get()); + } /** * Increment a specific gauge. @@ -603,4 +621,149 @@ public String toString() { return sb.toString(); } } + + /** + * Create a stream output statistics instance. + * @return the new instance + */ + + OutputStreamStatistics newOutputStreamStatistics() { + return new OutputStreamStatistics(); + } + + /** + * Merge in the statistics of a single output stream into + * the filesystem-wide statistics. + * @param statistics stream statistics + */ + private void mergeOutputStreamStatistics(OutputStreamStatistics statistics) { + incrementCounter(STREAM_WRITE_TOTAL_TIME, statistics.totalUploadDuration()); + incrementCounter(STREAM_WRITE_QUEUE_DURATION, statistics.queueDuration); + incrementCounter(STREAM_WRITE_TOTAL_DATA, statistics.bytesUploaded); + incrementCounter(STREAM_WRITE_BLOCK_UPLOADS, + statistics.blockUploadsCompleted); + + } + + /** + * Statistics updated by an output stream during its actual operation. + * Some of these stats may be relayed. However, as block upload is + * spans multiple + */ + @InterfaceAudience.Private + @InterfaceStability.Unstable + public final class OutputStreamStatistics implements Closeable { + private final AtomicLong blocksSubmitted = new AtomicLong(0); + private final AtomicLong blocksInQueue = new AtomicLong(0); + private final AtomicLong blocksActive = new AtomicLong(0); + private final AtomicLong blockUploadsCompleted = new AtomicLong(0); + private final AtomicLong blockUploadsFailed = new AtomicLong(0); + private final AtomicLong bytesPendingUpload = new AtomicLong(0); + + private final AtomicLong bytesUploaded = new AtomicLong(0); + private final AtomicLong transferDuration = new AtomicLong(0); + private final AtomicLong queueDuration = new AtomicLong(0); + private final AtomicLong exceptionsInMultipartFinalize = new AtomicLong(0); + + /** + * Block is queued for upload. + */ + void blockUploadQueued(int blockSize) { + blocksSubmitted.incrementAndGet(); + blocksInQueue.incrementAndGet(); + bytesPendingUpload.addAndGet(blockSize); + incrementGauge(STREAM_WRITE_BLOCK_UPLOADS_PENDING, 1); + incrementGauge(STREAM_WRITE_BLOCK_UPLOADS_DATA_PENDING, blockSize); + } + + /** Queued block has been scheduled for upload. */ + void blockUploadStarted(long duration, int blockSize) { + queueDuration.addAndGet(duration); + blocksInQueue.decrementAndGet(); + blocksActive.incrementAndGet(); + incrementGauge(STREAM_WRITE_BLOCK_UPLOADS_PENDING, -1); + incrementGauge(STREAM_WRITE_BLOCK_UPLOADS_ACTIVE, 1); + } + + /** A block upload has completed. */ + void blockUploadCompleted(long duration, int blockSize) { + this.transferDuration.addAndGet(duration); + incrementGauge(STREAM_WRITE_BLOCK_UPLOADS_ACTIVE, -1); + blocksActive.decrementAndGet(); + blockUploadsCompleted.incrementAndGet(); + } + + /** A block upload has failed. */ + void blockUploadFailed(long transferDuration, int blockSize) { + this.transferDuration.addAndGet(transferDuration); + blocksActive.decrementAndGet(); + blockUploadsFailed.incrementAndGet(); + } + + /** Intermediate report of bytes uploaded. */ + void bytesTransferred(long byteCount) { + bytesUploaded.addAndGet(byteCount); + bytesPendingUpload.addAndGet(-byteCount); + incrementGauge(STREAM_WRITE_BLOCK_UPLOADS_DATA_PENDING, -byteCount); + } + + void exceptionInMultipartComplete() { + exceptionsInMultipartFinalize.incrementAndGet(); + } + + void exceptionInMultipartAbort() { + exceptionsInMultipartFinalize.incrementAndGet(); + } + + + /** + * Output stream has closed. + * Trigger merge in of all statistics not updated during operation. + */ + @Override + public void close() { + if (bytesPendingUpload.get() > 0) { + LOG.warn("Closing output stream statistics while data is still marked" + + " as pending upload"); + } + mergeOutputStreamStatistics(this); + } + + long averageQueueTime() { + return blocksSubmitted.get() > 0 ? + (queueDuration.get() / blocksSubmitted.get()) : 0; + } + + double effectiveBandwidth() { + double duration = totalUploadDuration()/1000.0; + return duration > 0 ? + (bytesUploaded.get() / duration) : 0; + } + + long totalUploadDuration() { + return queueDuration.get() + transferDuration.get(); + } + + @Override + public String toString() { + final StringBuilder sb = new StringBuilder( + "OutputStreamStatistics{"); + sb.append("blocksSubmitted=").append(blocksSubmitted); + sb.append(", blocksInQueue=").append(blocksInQueue); + sb.append(", blocksActive=").append(blocksActive); + sb.append(", blockUploadsCompleted=").append(blockUploadsCompleted); + sb.append(", blockUploadsFailed=").append(blockUploadsFailed); + sb.append(", bytesPendingUpload=").append(bytesPendingUpload); + sb.append(", bytesUploaded=").append(bytesUploaded); + sb.append(", transferDuration=").append(transferDuration).append(" ms"); + sb.append(", queueDuration=").append(queueDuration).append(" ms"); + sb.append(", averageQueueTime=").append(averageQueueTime()).append(" ms"); + sb.append(", totalUploadDuration=").append(totalUploadDuration()) + .append(" ms"); + sb.append(", effectiveBandwidth=").append(effectiveBandwidth()) + .append(" bytes/s"); + sb.append('}'); + return sb.toString(); + } + } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Statistic.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Statistic.java index 75ce63303dec4..141ff0c9f6b50 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Statistic.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Statistic.java @@ -114,7 +114,33 @@ public enum Statistic { STREAM_CLOSE_BYTES_READ("stream_bytes_read_in_close", "Count of bytes read when closing streams during seek operations."), STREAM_ABORT_BYTES_DISCARDED("stream_bytes_discarded_in_abort", - "Count of bytes discarded by aborting the stream"); + "Count of bytes discarded by aborting the stream"), + STREAM_WRITE_FAILURES("stream_write_failures", + "Count of stream write failures reported"), + STREAM_WRITE_BLOCK_UPLOADS("stream_write_block_uploads", + "Count of block/partition uploads completed"), + STREAM_WRITE_BLOCK_UPLOADS_ACTIVE("stream_write_block_uploads_active", + "Count of block/partition uploads completed"), + STREAM_WRITE_BLOCK_UPLOADS_COMMITTED("stream_write_block_uploads_committed", + "Count of number of block uploads committed"), + STREAM_WRITE_BLOCK_UPLOADS_ABORTED("stream_write_block_uploads_aborted", + "Count of number of block uploads aborted"), + + STREAM_WRITE_BLOCK_UPLOADS_PENDING("stream_write_block_uploads_pending", + "Gauge of block/partitions uploads queued to be written"), + STREAM_WRITE_BLOCK_UPLOADS_DATA_PENDING( + "stream_write_block_uploads_data_pending", + "Gauge of block/partitions data uploads queued to be written"), + STREAM_WRITE_TOTAL_TIME("stream_write_total_time", + "Count of total time taken for uploads to complete"), + STREAM_WRITE_TOTAL_DATA("stream_write_total_data", + "Count of total data uploaded in block output"), + STREAM_WRITE_BANDWIDTH("stream_write_bandwidth", + "Gauge of inferred bandwidth"), + STREAM_WRITE_QUEUE_DURATION("stream_write_queue_duration", + "Total queue duration of all block uploads"), + + ; private static final Map SYMBOL_MAP = new HashMap<>(Statistic.values().length); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/AbstractSTestS3AHugeFiles.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/AbstractSTestS3AHugeFiles.java index 750ad840f15b9..d06ad678f4676 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/AbstractSTestS3AHugeFiles.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/AbstractSTestS3AHugeFiles.java @@ -248,9 +248,6 @@ public void test_040_PositionedReadHugeFile() throws Throwable { filetype, mb); LOG.info("Time per positioned read = {} nS", toHuman(timer.nanosPerOperation(ops))); - final long difference = readAtEOF.duration() - readAtByte0Again.duration(); - LOG.info("Difference between read at start & end of {} is {} nS", - filetype, toHuman(difference)); } @Test @@ -290,8 +287,8 @@ public void test_100_renameHugeFile() throws Throwable { fs.rename(hugefile, hugefileRenamed); long mb = Math.max(filesize / _1MB, 1); timer.end("Time to rename file of %d MB", mb); - LOG.info("Time per MB to rename = {} nS", toHuman(timer.duration() / mb)); - LOG.info("Effective Bandwidth: {} MB/s" , timer.bandwidth(filesize)); + LOG.info("Time per MB to rename = {} nS", + toHuman(timer.nanosPerOperation(mb))); logFSState(); S3AFileStatus destFileStatus = fs.getFileStatus(hugefileRenamed); assertEquals(filesize, destFileStatus.getLen()); @@ -300,6 +297,8 @@ public void test_100_renameHugeFile() throws Throwable { ContractTestUtils.NanoTimer timer2 = new ContractTestUtils.NanoTimer(); fs.rename(hugefileRenamed, hugefile); timer2.end("Renaming back"); + LOG.info("Time per MB to rename = {} nS", + toHuman(timer2.nanosPerOperation(mb))); } @Test From 6c869f61cdc4277db6b4cf6079e25ff107fa6f43 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Mon, 12 Sep 2016 12:30:09 +0100 Subject: [PATCH 18/20] add block options to core-default.xml and write big section documenting the feature. Minor cleanups of code --- .../src/main/resources/core-default.xml | 29 +++ .../apache/hadoop/fs/s3a/S3AFileSystem.java | 3 +- .../apache/hadoop/fs/s3a/S3AOutputStream.java | 5 - .../site/markdown/tools/hadoop-aws/index.md | 239 +++++++++++++++++- 4 files changed, 267 insertions(+), 9 deletions(-) diff --git a/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml b/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml index 0888b5a953bb0..9b8c229d6719a 100644 --- a/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml +++ b/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml @@ -1101,6 +1101,34 @@ uploading (fs.s3a.threads.max) or queueing (fs.s3a.max.total.tasks) + + fs.s3a.block.upload + false + + Use the incremental block upload mechanism with + the buffering mechanism set in fs.s3a.block.output.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. + + + + + fs.s3a.block.output.buffer + disk + + Buffering mechanims to use when using S3a Block upload + (fs.s3a.block.upload=true). values: disk, array. + "disk" will use the directories listed in fs.s3a.buffer.dir as + the location(s) to save data prior to being uploaded. + If using "array", the data is stored in memory arrays. + The fs.s3a.threads.max and fs.s3a.max.total.tasks values must then + be kept low so as to prevent heap overflows. + No effect if fs.s3a.block.upload is false. + + + fs.s3a.readahead.range 65536 @@ -1116,6 +1144,7 @@ upload. No effect if fs.s3a.fast.upload is false. + fs.s3a.user.agent.prefix diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java index f1d5a0d00e710..3cf39ce8e30ad 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java @@ -133,8 +133,7 @@ public class S3AFileSystem extends FileSystem { private boolean fastUploadEnabled; private boolean blockUploadEnabled; private String blockOutputBuffer; - private S3ADataBlocks.AbstractBlockFactory - blockFactory; + private S3ADataBlocks.AbstractBlockFactory blockFactory; /** Called after a new FileSystem instance is constructed. * @param name a uri whose authority section names the host, port, etc. diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AOutputStream.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AOutputStream.java index 1532c46c82436..1b1612c20dc0f 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AOutputStream.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AOutputStream.java @@ -50,8 +50,6 @@ public class S3AOutputStream extends OutputStream { private boolean closed; private final String key; private final Progressable progress; - private final long partSize; - private final long partSizeThreshold; private final S3AFileSystem fs; private final LocalDirAllocator lDirAlloc; @@ -64,9 +62,6 @@ public S3AOutputStream(Configuration conf, this.progress = progress; this.fs = fs; - partSize = fs.getPartitionSize(); - partSizeThreshold = fs.getMultiPartThreshold(); - lDirAlloc = fs.getDirectoryAllocator(); backupFile = lDirAlloc.createTmpFileForWrite("output-", diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md index 41cd398f3a09f..f2fe881740532 100644 --- a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md +++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md @@ -881,8 +881,237 @@ 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** +### Improving S3A Upload Performance/Reducing Delays + +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 has time implications: + +* 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: + +* Application code often assumes that the `close()` call is fast and so does not +need to be executed in a separate thread: the delays can create bottlenecks in +operations. + +* Very slow uploads sometimes cause applications to time out. + +* Large uploads may consume more disk capacity than is available in +the temporary disk. + +There is ongoing work to address these issues, primarily through +supporting incremental uploads of large objects, uploading intermediate +blocks of data (in S3 terms, "parts of a multipart upload). This can make more +effective use of bandwidth during the write sequence, and may reduce the delay +during the final `close()` call to that needed to upload *all outstanding +data*, rather than *all data generated.* + +This work 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). + +#### Stabilizing: S3ABlockOutputStream + + +**Warning: NEW in hadoop 2.9. UNSTABLE: use at own risk** + +This is high-performance incremental file upload mechanism 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. Uploads blocks in parallel in background threads. +1. Begins uploading blocks as soon as the buffered data exceeds this partition + size. +1. Can buffer data to disk (default) or in memory. +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. + +```xml + + fs.s3a.block.upload + false + + Use the incremental block upload mechanism with + the buffering mechanism set in fs.s3a.block.output.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. + + + + + fs.s3a.block.output.buffer + disk + + Buffering mechanims to use when using S3a Block upload + (fs.s3a.block.upload=true). values: disk, array. + "disk" will use the directories listed in fs.s3a.buffer.dir as + the location(s) to save data prior to being uploaded. + If using "array", the data is stored in memory arrays. + The fs.s3a.threads.max and fs.s3a.max.total.tasks values must then + be kept low so as to prevent heap overflows. + No effect if fs.s3a.block.upload is false. + + +``` + +**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. + + +##### Disk upload `fs.s3a.block.output.buffer=disk` + +When `fs.s3a.block.output.buffer` is set to `array`, all data is buffered +to local hard disks in 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.block.output=false` and `fs.s3a.fast.upload=false`. + +Unlike the original buffering, uploads are incremental during the write operation: +once the size of the output stream exceeds the number of bytes set in `fs.s3a.multipart.size`, +the upload is queued to a background thread. + +This can make more effective use of available bandwidth to upload data +to S3, and may reduce delays in the `OutputStream.close()` operation. + +The available upload bandwidth is generally limited by network capacity — +rather than the number of threads performing uploads. Having a large value +of `fs.s3a.threads.max` does not generally reduce upload time or increase +throughput. Instead the bandwidth is shared between all threads, slowing them +all down. We recommend a low number of threads, especially when uploading +over long-haul connections. + +```xml + + fs.s3a.threads.max + 5 + Maximum number of concurrent active (part)uploads, + which each use a thread from the threadpool. + +``` +Because the upload data is buffered via disk, there is little risk of heap +overflow as the queue of pending uploads (limited by `fs.s3a.max.total.tasks`) +increases. + +```xml + + fs.s3a.max.total.tasks + 20 + Number of (part)uploads allowed to the queue before + blocking additional uploads. + +``` + + +##### Array upload: `fs.s3a.block.output.buffer=array` + +When `fs.s3a.block.output.buffer` is set to `array`, all data is buffered +in memory prior to upload. This *may* be faster than buffering to disk. + +This `array` option is similar to the in-memory-only stream implemented in +`S3AFastOutputStream`: the amount of data which can be buffered is +limited by available heap. The slower the write bandwidth to S3, the greater +the risk of heap overflows. To reduce the risk of this, the number of +threads performing uploads and the size of queued upload operations must be +kept low. As an example: three threads and one queued task will consume +`(3 + 1) * fs.s3a.multipart.size` bytes. + +```xml + + fs.s3a.threads.max + 3 + Maximum number of concurrent active (part)uploads, + which each use a thread from the threadpool. + + + + fs.s3a.max.total.tasks + 1 + Number of (part)uploads allowed to the queue before + blocking additional uploads. + +``` + +#### 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 —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 +upload data from operations which are considered to have failed by virtue +of having been in progress for longer than this purge time. + +```xml + + fs.s3a.multipart.purge + true + True if you want to purge existing multipart uploads that may not have been + completed/aborted correctly + + + + fs.s3a.multipart.purge.age + 86400 + Minimum age in seconds of multipart uploads to purge + +``` + +If an S3A client is instantited with `fs.s3a.multipart.purge=true`, +it sets the purge time *for the entire bucket*. That is: it will affect all +multipart uploads to that bucket, from all applications —and persist until set +to a a new value. Leaving `fs.s3a.multipart.purge` to its default, `false`, +means that the client will not make any attempt to reset or change the partition +rate. + +The best practise for using this option is for it to be explicitly set in a +bucket management process to a value which is known to be greater than the +maximum duration of any multipart write, while leaving the configuration option to its +default, `false` in `core-site.xml`. This eliminates the risk +that one client unintentionally sets a purge time so low as to break multipart +uploads being attempted by other applications. + + +#### S3AFastOutputStream + +**Warning: NEW in hadoop 2.7. UNSTABLE, EXPERIMENTAL: use at own risk** + +**Hadoop 2.9+: use the block output mechanism instead** + + fs.s3a.fast.upload @@ -916,6 +1145,12 @@ the available memory. These settings should be tuned to the envisioned workflow (some large files, many small ones, ...) and the physical limitations of the machine and cluster (memory, network bandwidth). + +**Note** + +Consult the section *Cleaning up After Incremental Upload Failures* for +advice on how to manage multipart upload failures. + ### S3A Experimental "fadvise" input policy support **Warning: EXPERIMENTAL: behavior may change in future** From 21cc2ce699f9fa4506a10d4e955356a1b076f305 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Tue, 20 Sep 2016 20:11:12 +0100 Subject: [PATCH 19/20] HADOOP-13560 big iteration on this -supercede the Fast output stream, -run tests, tune outcomes (especially race conditions in multipart operations) --- .../src/main/resources/core-default.xml | 48 +-- hadoop-tools/hadoop-aws/pom.xml | 6 +- .../org/apache/hadoop/fs/s3a/Constants.java | 19 +- .../hadoop/fs/s3a/S3ABlockOutputStream.java | 117 ++++-- .../apache/hadoop/fs/s3a/S3ADataBlocks.java | 360 ++++++++++++++++-- .../apache/hadoop/fs/s3a/S3AFileSystem.java | 53 +-- .../hadoop/fs/s3a/S3AInstrumentation.java | 1 + .../site/markdown/tools/hadoop-aws/index.md | 172 +++++++-- .../contract/s3a/ITestS3AContractDistCp.java | 10 +- .../fs/s3a/ITestS3ABlockingThreadPool.java | 6 +- ... ITestS3AEncryptionBlockOutputStream.java} | 9 +- .../apache/hadoop/fs/s3a/S3ATestUtils.java | 8 +- .../apache/hadoop/fs/s3a/TestDataBlocks.java | 123 ++++++ .../s3a/scale/AbstractSTestS3AHugeFiles.java | 3 - .../fs/s3a/scale/STestS3AHugeDistCP.java | 32 -- ... => STestS3AHugeFilesByteArrayBlocks.java} | 11 +- .../scale/STestS3AHugeFilesDiskBlocks.java | 4 - .../scale/STestS3AHugeFilesMemoryBlocks.java | 1 - 18 files changed, 783 insertions(+), 200 deletions(-) rename hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/{ITestS3AEncryptionFastOutputStream.java => ITestS3AEncryptionBlockOutputStream.java} (78%) create mode 100644 hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestDataBlocks.java delete mode 100644 hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/STestS3AHugeDistCP.java rename hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/{STestS3AHugeFilesFastUpload.java => STestS3AHugeFilesByteArrayBlocks.java} (71%) diff --git a/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml b/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml index 9b8c229d6719a..95e9d61b28bb0 100644 --- a/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml +++ b/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml @@ -1056,13 +1056,21 @@ fs.s3a.multipart.purge false True if you want to purge existing multipart uploads that may not have been - completed/aborted correctly + completed/aborted correctly. The corresponding purge age is defined in + fs.s3a.multipart.purge.age. + If set, when the filesystem is instantiated then all outstanding uploads + older than the purge age will be terminated -across the entire bucket. + This will impact multipart uploads by other applications and users. so should + be used sparingly, with an age value chosen to stop failed uploads, without + breaking ongoing operations. + fs.s3a.multipart.purge.age 86400 - Minimum age in seconds of multipart uploads to purge + Minimum age in seconds of multipart uploads to purge. + @@ -1092,15 +1100,6 @@ uploads to. - - fs.s3a.fast.upload - false - 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) - - fs.s3a.block.upload false @@ -1118,13 +1117,22 @@ fs.s3a.block.output.buffer disk - Buffering mechanims to use when using S3a Block upload - (fs.s3a.block.upload=true). values: disk, array. + Buffering mechanism to use when using S3A Block upload + (fs.s3a.block.upload=true). Values: disk, array, bytebuffer. + "disk" will use the directories listed in fs.s3a.buffer.dir as the location(s) to save data prior to being uploaded. - If using "array", the data is stored in memory arrays. - The fs.s3a.threads.max and fs.s3a.max.total.tasks values must then - be kept low so as to prevent heap overflows. + + "array" uses arrays in the JVM heap + + "bytebuffer" uses off-heap memory within the JVM. + + Both "array" and "bytebuffer" will consume memory proportional + to the partition size * active+pending block uploads. + + To avoid running out of memory, keep the fs.s3a.threads.max and + fs.s3a.max.total.tasks values low. + No effect if fs.s3a.block.upload is false. @@ -1137,14 +1145,6 @@ any call to setReadahead() is made to an open stream. - - fs.s3a.fast.buffer.size - 1048576 - Size of initial memory buffer in bytes allocated for an - upload. No effect if fs.s3a.fast.upload is false. - - - fs.s3a.user.agent.prefix diff --git a/hadoop-tools/hadoop-aws/pom.xml b/hadoop-tools/hadoop-aws/pom.xml index 70860a2f63649..ce5b214830546 100644 --- a/hadoop-tools/hadoop-aws/pom.xml +++ b/hadoop-tools/hadoop-aws/pom.xml @@ -169,7 +169,6 @@ **/ITestJets3tNativeS3FileSystemContract.java **/ITestS3ABlockingThreadPool.java - **/ITestS3AFastOutputStream.java **/ITestS3AFileSystemContract.java **/ITestS3AMiniYarnCluster.java **/ITest*Root*.java @@ -228,6 +227,11 @@ verify + + + ${fs.s3a.scale.test.huge.filesize} + ${fs.s3a.scale.test.timeout} + 3600 diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java index 1cefb73c1701d..6e1123d3a4547 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java @@ -32,11 +32,12 @@ @InterfaceStability.Evolving public final class Constants { - public static final int MULTIPART_MIN_SIZE = 5 * 1024 * 1024; - private Constants() { } + /** The minimum multipart size which S3 supports. */ + public static final int MULTIPART_MIN_SIZE = 5 * 1024 * 1024; + // s3 access key public static final String ACCESS_KEY = "fs.s3a.access.key"; @@ -132,10 +133,12 @@ private Constants() { public static final String BUFFER_DIR = "fs.s3a.buffer.dir"; // should we upload directly from memory rather than using a file buffer + @Deprecated public static final String FAST_UPLOAD = "fs.s3a.fast.upload"; public static final boolean DEFAULT_FAST_UPLOAD = false; //initial size of memory buffer for a fast upload + @Deprecated public static final String FAST_BUFFER_SIZE = "fs.s3a.fast.buffer.size"; public static final int DEFAULT_FAST_BUFFER_SIZE = 1048576; //1MB @@ -152,7 +155,7 @@ private Constants() { // purge any multipart uploads older than this number of seconds public static final String PURGE_EXISTING_MULTIPART_AGE = "fs.s3a.multipart.purge.age"; - public static final long DEFAULT_PURGE_EXISTING_MULTIPART_AGE = 14400; + public static final long DEFAULT_PURGE_EXISTING_MULTIPART_AGE = 86400; // s3 server-side encryption public static final String SERVER_SIDE_ENCRYPTION_ALGORITHM = @@ -189,7 +192,7 @@ private Constants() { "fs.s3a.block.output"; /** * What buffer to use. - * Default is {@link #OUTPUT_INCREMENTAL_BUFFER_DISK} + * Default is {@link #BLOCK_OUTPUT_BUFFER_DISK} * Value: {@value} */ @InterfaceStability.Unstable @@ -212,6 +215,14 @@ private Constants() { public static final String BLOCK_OUTPUT_BUFFER_ARRAY = "array"; + /** + * Use a byte buffer. May be more memory efficient than the + * {@link #BLOCK_OUTPUT_BUFFER_ARRAY}: {@value}. + */ + @InterfaceStability.Unstable + public static final String BLOCK_OUTPUT_BYTEBUFFER = + "bytebuffer"; + /** * Default buffer option: {@value}. */ diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java index 9230652846f94..2212d3a91e6b1 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java @@ -74,29 +74,60 @@ class S3ABlockOutputStream extends OutputStream { private static final Logger LOG = LoggerFactory.getLogger( S3ABlockOutputStream.class); + /** Owner fs. */ private final S3AFileSystem fs; + + /** Object being uploaded. */ private final String key; + + /** Bucket; used in building requests. */ private final String bucket; + + /** Size of all blocks. */ private final int blockSize; + + /** optional hard-coded ACL. */ private final CannedAccessControlList cannedACL; + + /** Callback for progress. */ private final ProgressListener progressListener; private final ListeningExecutorService executorService; - private final RetryPolicy retryPolicy; + + /** Retry policy for multipart commits; not all AWS SDK versions retry */ + private final RetryPolicy retryPolicy = + RetryPolicies.retryUpToMaximumCountWithProportionalSleep( + 5, + 2000, + TimeUnit.MILLISECONDS); + /** + * Factory for blocks. + */ private final S3ADataBlocks.AbstractBlockFactory 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 volatile boolean closed; + + /** Current data block. Null means none currently active */ private S3ADataBlocks.DataBlock currentBlock; + + /** Count of blocks uploaded. */ private long blockCount = 0; - private final S3AInstrumentation.OutputStreamStatistics statistics; + /** Statistics to build up. */ + private final S3AInstrumentation.OutputStreamStatistics statistics; /** - * S3A output stream which uploads blocks as soon as there is enough - * data. + * An S3A output stream which uploads partitions in a separate pool of + * threads; different {@link S3ADataBlocks.AbstractBlockFactory} + * instances can control where data is buffered. * * @param fs S3AFilesystem - * @param key S3 key name + * @param key S3 object to work on. * @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. @@ -127,13 +158,19 @@ class S3ABlockOutputStream extends OutputStream { this.progressListener = (progress instanceof ProgressListener) ? (ProgressListener) progress : new ProgressableListener(progress); - this.retryPolicy = RetryPolicies.retryUpToMaximumCountWithProportionalSleep( - 5, 2000, TimeUnit.MILLISECONDS); LOG.debug("Initialized S3AFastOutputStream for bucket '{}' key '{}' " + "output to {}", bucket, key, currentBlock); maybeCreateDestStream(); } + /** + * Get the S3 Client to talk to. + * @return the S3Client instance of the owner FS. + */ + private AmazonS3 getS3Client() { + return fs.getAmazonS3Client(); + } + /** * Demand create a destination stream. * @throws IOException on any failure to create @@ -219,22 +256,27 @@ public synchronized void write(byte[] source, int offset, int len) } /** - * Trigger the upload. + * Start an asynchronous upload of the current block. * @throws IOException Problems opening the destination for upload * or initializing the upload. */ private synchronized void uploadCurrentBlock() throws IOException { + Preconditions.checkNotNull(currentBlock, "current block"); LOG.debug("Writing block # {}", blockCount); if (multiPartUpload == null) { multiPartUpload = initiateMultiPartUpload(); } multiPartUpload.uploadBlockAsync(currentBlock); + // close the block currentBlock.close(); + // set it to null, so the next write will create a new block. currentBlock = null; } /** - * Close the stream. This will not return until the upload is complete + * Close the stream. + * + * This will not return until the upload is complete * or the attempt to perform the upload has failed. * Exceptions raised in this method are indicative that the write has * failed and data is at risk of being lost. @@ -246,10 +288,10 @@ public synchronized void close() throws IOException { return; } closed = true; - LOG.debug("{}: Closing block #{}: {}, data to upload = {}", + LOG.debug("{}: Closing block #{}: current block= {}, data to upload = {}", this, blockCount, - currentBlock, + currentBlock == null ? "(none)" : currentBlock, currentBlock == null ? 0 : currentBlock.dataSize() ); try { if (multiPartUpload == null) { @@ -307,7 +349,9 @@ private MultiPartUpload initiateMultiPartUpload() throws IOException { } /** - * Upload the current block as a single PUT request. + * Upload the current block as a single PUT request; if the buffer + * is empty a 0-byte PUT will be invoked, as it is needed to create an + * entry at the far end. * @throws IOException any problem. */ private void putObject() throws IOException { @@ -349,10 +393,6 @@ public PutObjectResult call() throws Exception { } } - private AmazonS3 getS3Client() { - return fs.getAmazonS3Client(); - } - @Override public String toString() { final StringBuilder sb = new StringBuilder( @@ -371,6 +411,10 @@ private void incrementWriteOperations() { fs.incrementWriteOperations(); } + /** + * Current time in nanoseconds. + * @return time + */ private long now() { return System.currentTimeMillis(); } @@ -384,7 +428,7 @@ private class MultiPartUpload { public MultiPartUpload(String uploadId) { this.uploadId = uploadId; - this.partETagsFutures = new ArrayList<>(); + this.partETagsFutures = new ArrayList<>(2); LOG.debug("Initiated multi-part upload for bucket '{}' key '{}' with " + "id '{}'", bucket, key, uploadId); } @@ -401,13 +445,17 @@ private void uploadBlockAsync(final S3ADataBlocks.DataBlock block) final InputStream uploadStream = block.openForUpload(); final int currentPartNumber = partETagsFutures.size() + 1; final UploadPartRequest request = - new UploadPartRequest().withBucketName(bucket).withKey(key) - .withUploadId(uploadId).withInputStream(uploadStream) - .withPartNumber(currentPartNumber).withPartSize(size); + new UploadPartRequest() + .withBucketName(bucket) + .withKey(key) + .withUploadId(uploadId) + .withInputStream(uploadStream) + .withPartNumber(currentPartNumber) + .withPartSize(size); long transferQueueTime = now(); BlockUploadProgress callback = new BlockUploadProgress( - block, progressListener, transferQueueTime); + block, progressListener, transferQueueTime); request.setGeneralProgressListener(callback); statistics.blockUploadQueued(block.dataSize()); ListenableFuture partETagFuture = @@ -430,7 +478,12 @@ public PartETag call() throws Exception { partETagsFutures.add(partETagFuture); } - private List waitForAllPartUploads() throws IOException { + /** + * Block awating all outstanding uploads to complete. + * @return list of results + * @throws IOException IO Problems + */ + private List waitForAllPartUploads() throws IOException { LOG.debug("Waiting for {} uploads to complete", partETagsFutures.size()); try { return Futures.allAsList(partETagsFutures).get(); @@ -541,16 +594,23 @@ private boolean shouldRetry(AmazonClientException e, int retryCount) { */ private class BlockUploadProgress implements ProgressListener { private final S3ADataBlocks.DataBlock block; - private final ProgressListener next; + private final ProgressListener nextListener; private final long transferQueueTime; private long transferStartTime; + /** + * Track the progress of a single block upload. + * @param block block to monitor + * @param nextListener optional next progress listener + * @param transferQueueTime time the block was transferred + * into the queue + */ private BlockUploadProgress(S3ADataBlocks.DataBlock block, - ProgressListener next, + ProgressListener nextListener, long transferQueueTime) { this.block = block; this.transferQueueTime = transferQueueTime; - this.next = next; + this.nextListener = nextListener; } @Override @@ -585,12 +645,15 @@ public void progressChanged(ProgressEvent progressEvent) { // nothing } - if (next != null) { - next.progressChanged(progressEvent); + if (nextListener != null) { + nextListener.progressChanged(progressEvent); } } } + /** + * Bridge from AWS {@code ProgressListener} to Hadoop {@link Progressable}. + */ private static class ProgressableListener implements ProgressListener { private final Progressable progress; diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ADataBlocks.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ADataBlocks.java index fa97a873939c7..15e8c2d73c5ed 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ADataBlocks.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ADataBlocks.java @@ -19,6 +19,8 @@ package org.apache.hadoop.fs.s3a; import com.google.common.base.Preconditions; +import org.apache.hadoop.fs.FSExceptionMessages; +import org.apache.hadoop.util.DirectBufferPool; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -26,12 +28,16 @@ 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.IOException; import java.io.InputStream; +import java.nio.BufferUnderflowException; +import java.nio.ByteBuffer; +import java.util.concurrent.atomic.AtomicInteger; import static org.apache.hadoop.fs.s3a.S3ADataBlocks.DataBlock.DestState.*; @@ -104,6 +110,9 @@ DataBlock create(int limit) throws IOException { } } + /** + * This represents a block being uploaded. + */ static abstract class DataBlock implements Closeable { enum DestState {Writing, Upload, Closed} @@ -116,9 +125,14 @@ protected synchronized void enterState(DestState current, DestState next) { state = next; } - protected void verifyState(DestState current) { - Preconditions.checkState(state == current, - "Expected stream state " + current + " -but actual state is " + state + /** + * Verify that the block is in the declared state. + * @param expected expected state. + * @throws IllegalStateException if the DataBlock is in the wrong state + */ + protected void verifyState(DestState expected) { + Preconditions.checkState(state == expected, + "Expected stream state " + expected + " -but actual state is " + state + " in " + this); } @@ -132,38 +146,64 @@ DestState getState() { */ 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); /** - * Is there data in the block. + * 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 blokc 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: - * @param b buffer - * @param off offset - * @param len length of write + * 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 b[], int off, int len) throws IOException { + 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 { } /** * 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 */ @@ -172,14 +212,6 @@ InputStream openForUpload() throws IOException { return null; } - /** - * actions to take on block upload completion. - * @throws IOException Any failure - */ - void blockUploadCompleted() throws IOException { - - } - /** * Enter the closed state. * @return true if the class was in any other state, implying that @@ -210,7 +242,7 @@ static class ByteArrayBlock extends DataBlock { // cache data size so that it is consistent after the buffer is reset. private Integer dataSize; - public ByteArrayBlock(int limit) { + ByteArrayBlock(int limit) { this.limit = limit; buffer = new ByteArrayOutputStream(); } @@ -236,20 +268,20 @@ InputStream openForUpload() throws IOException { } @Override - public boolean hasCapacity(long bytes) { + boolean hasCapacity(long bytes) { return dataSize() + bytes <= limit; } @Override - public int remainingCapacity() { + int remainingCapacity() { return limit - dataSize(); } @Override - int write(byte[] b, int off, int len) throws IOException { - super.write(b, off, len); + int write(byte[] b, int offset, int len) throws IOException { + super.write(b, offset, len); int written = Math.min(remainingCapacity(), len); - buffer.write(b, off, written); + buffer.write(b, offset, written); return written; } @@ -260,17 +292,283 @@ public void close() throws IOException { buffer = null; } } + } + + /** + * 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 FileBlock} buffer delegates the work of deleting files to + * the {@link FileBlock.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 AbstractBlockFactory { + + private final DirectBufferPool bufferPool =new DirectBufferPool(); + private final AtomicInteger buffersOutstanding = new AtomicInteger(0); + + @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() { final StringBuilder sb = new StringBuilder( - "ByteArrayBlock{"); - sb.append("state=").append(getState()); - sb.append(", dataSize=").append(dataSize()); - sb.append(", limit=").append(limit); + "ByteBufferBlockFactory{"); + sb.append(", buffersOutstanding=").append(buffersOutstanding.toString()); sb.append('}'); return sb.toString(); } + + /** + * 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 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 openForUpload() throws IOException { + super.openForUpload(); + 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 + public void close() throws IOException { + if (enterClosedState()) { + LOG.debug("Closed {}", this); + buffer = null; + } + } + + @Override + public String toString() { + final StringBuilder sb = new StringBuilder( + "ByteBufferBlock{"); + sb.append("state=").append(getState()); + sb.append(", dataSize=").append(dataSize()); + sb.append(", limit=").append(bufferSize); + sb.append(", remainingCapacity=").append(remainingCapacity()); + sb.append('}'); + return sb.toString(); + } + + } + + /** + * 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; + } + + @Override + public synchronized void close() throws IOException { + 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 int read() throws IOException { + verifyOpen(); + try { + return byteBuffer.get() & 0xFF; + } catch (BufferUnderflowException e) { + 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 int available() { + Preconditions.checkState(byteBuffer != null, + FSExceptionMessages.STREAM_IS_CLOSED); + return byteBuffer.remaining(); + } + + /** + * Get the current buffer position. + * @return the buffer position + */ + public int position() { + return byteBuffer.position(); + } + + /** + * Check if there is data left. + * @return true if there is data remaining in the buffer. + */ + public 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. + */ + public 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); + sb.append(", byteBuffer=").append(byteBuffer); + if (byteBuffer != null) { + sb.append(", available=").append(available()); + } + sb.append('}'); + return sb.toString(); + } + } } /** @@ -311,10 +609,10 @@ int remainingCapacity() { } @Override - int write(byte[] b, int off, int len) throws IOException { - super.write(b, off, len); + int write(byte[] b, int offset, int len) throws IOException { + super.write(b, offset, len); int written = Math.min(remainingCapacity(), len); - out.write(b, off, written); + out.write(b, offset, written); bytesWritten += written; return written; } @@ -474,6 +772,8 @@ static AbstractBlockFactory createFactory(String name) { return new MemoryBlockFactory(); case Constants.BLOCK_OUTPUT_BUFFER_DISK: return new DiskBlockFactory(); + case Constants.BLOCK_OUTPUT_BYTEBUFFER: + return new ByteBufferBlockFactory(); default: throw new IllegalArgumentException("Unsupported block buffer" + " \"" + name + "\""); diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java index 3cf39ce8e30ad..6495f1db096ef 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java @@ -125,16 +125,28 @@ public class S3AFileSystem extends FileSystem { private S3AStorageStatistics storageStatistics; private long readAhead; private S3AInputPolicy inputPolicy; - private static final AtomicBoolean warnedOfCoreThreadDeprecation = - 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 fastUploadEnabled; private boolean blockUploadEnabled; private String blockOutputBuffer; private S3ADataBlocks.AbstractBlockFactory blockFactory; + /* + * Register Deprecated options. + */ + static { + Configuration.addDeprecations(new Configuration.DeprecationDelta[]{ + new Configuration.DeprecationDelta("fs.s3a.threads.core", + null, + "Unsupported option \"fs.s3a.threads.core\" will be ignored"), + new Configuration.DeprecationDelta(FAST_UPLOAD, + BLOCK_OUTPUT, + FAST_UPLOAD + " has been replaced by " + BLOCK_OUTPUT + + " with multiple buffering options. Switching to block output.") + }); + } + /** Called after a new FileSystem instance is constructed. * @param name a uri whose authority section names the host, port, etc. * for this FileSystem @@ -179,13 +191,7 @@ public StorageStatistics provide() { } }); - if (conf.get("fs.s3a.threads.core") != null && - warnedOfCoreThreadDeprecation.compareAndSet(false, true)) { - LoggerFactory.getLogger( - "org.apache.hadoop.conf.Configuration.deprecation") - .warn("Unsupported option \"fs.s3a.threads.core\"" + - " will be ignored {}", conf.get("fs.s3a.threads.core")); - } + int maxThreads = conf.getInt(MAX_THREADS, DEFAULT_MAX_THREADS); if (maxThreads < 2) { LOG.warn(MAX_THREADS + " must be at least 2: forcing to 2."); @@ -218,21 +224,15 @@ public StorageStatistics provide() { conf.getTrimmed(INPUT_FADVISE, INPUT_FADV_NORMAL)); blockUploadEnabled = conf.getBoolean(BLOCK_OUTPUT, false); + blockOutputBuffer = conf.getTrimmed(BLOCK_OUTPUT_BUFFER, + DEFAULT_BLOCK_OUTPUT_BUFFER); + if (blockUploadEnabled) { - blockOutputBuffer = conf.getTrimmed(BLOCK_OUTPUT_BUFFER, - DEFAULT_BLOCK_OUTPUT_BUFFER); partSize = ensureOutputParameterInRange(MULTIPART_SIZE, partSize); blockFactory = S3ADataBlocks.createFactory(blockOutputBuffer); blockFactory.init(this); LOG.debug("Uploading data via Block Upload, buffer = {}", blockOutputBuffer); - } else { - fastUploadEnabled = getConf().getBoolean(FAST_UPLOAD, - DEFAULT_FAST_UPLOAD); - if (fastUploadEnabled) { - partSize = ensureOutputParameterInRange(MULTIPART_SIZE, partSize); - - } } } catch (AmazonClientException e) { throw translateException("initializing ", new Path(name), e); @@ -549,18 +549,6 @@ public FSDataOutputStream create(Path f, FsPermission permission, blockFactory, instrumentation.newOutputStreamStatistics()), null); - } else if (fastUploadEnabled) { - output = new FSDataOutputStream( - new S3AFastOutputStream(s3, - this, - bucket, - key, - progress, - cannedACL, - partSize, - multiPartThreshold, - threadPoolExecutor), - statistics); } else { // We pass null to FSDataOutputStream so it won't count writes that @@ -1805,7 +1793,6 @@ public String toString() { sb.append(", workingDir=").append(workingDir); sb.append(", inputPolicy=").append(inputPolicy); sb.append(", partSize=").append(partSize); - sb.append(", fastUpload=").append(fastUploadEnabled); sb.append(", enableMultiObjectsDelete=").append(enableMultiObjectsDelete); sb.append(", maxKeys=").append(maxKeys); if (cannedACL != null) { @@ -1820,7 +1807,7 @@ public String toString() { .append('\''); } if (blockFactory != null) { - sb.append("Block Upload via ").append(blockFactory); + sb.append("Block Upload enabled via ").append(blockFactory); } sb.append(", statistics {") .append(statistics) diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java index 15ce2bc7219b7..d86694c9c87b1 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java @@ -118,6 +118,7 @@ public class S3AInstrumentation { private static final Statistic[] GAUGES_TO_CREATE = { OBJECT_PUT_REQUESTS_ACTIVE, OBJECT_PUT_BYTES_PENDING, + STREAM_WRITE_BLOCK_UPLOADS_PENDING, STREAM_WRITE_BLOCK_UPLOADS_DATA_PENDING, STREAM_WRITE_BANDWIDTH, }; diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md index f2fe881740532..4b117ab663107 100644 --- a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md +++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md @@ -928,7 +928,15 @@ has continued with ` S3ABlockOutputStream` #### Stabilizing: S3ABlockOutputStream -**Warning: NEW in hadoop 2.9. UNSTABLE: use at own risk** +**New in Hadoop 2.9+; replaces the "fast" upload of Hadoop 2.7** + +The default mechanism for file uploads in S3 buffers the entire output to +a file, then uploads it in the final `OutputStream.close()` call, a call +which then takes time proportional to the data to upload. + +The Block output stream, `S3ABlockOutputStream` writes the file in blocks +as the amount of data written exceeds the partition size defined in +`fs.s3a.multipart.size`. This is high-performance incremental file upload mechanism which: @@ -942,12 +950,14 @@ This is high-performance incremental file upload mechanism which: 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. +1. Generates output statistics as metrics on the filesystem, including + statistics of active and pending block uploads. + ```xml fs.s3a.block.upload - false + true Use the incremental block upload mechanism with the buffering mechanism set in fs.s3a.block.output.buffer. @@ -962,15 +972,33 @@ This is high-performance incremental file upload mechanism which: fs.s3a.block.output.buffer disk - Buffering mechanims to use when using S3a Block upload - (fs.s3a.block.upload=true). values: disk, array. + Buffering mechanism to use when using S3A Block upload + (fs.s3a.block.upload=true). Values: disk, array, bytebuffer. + "disk" will use the directories listed in fs.s3a.buffer.dir as the location(s) to save data prior to being uploaded. - If using "array", the data is stored in memory arrays. - The fs.s3a.threads.max and fs.s3a.max.total.tasks values must then - be kept low so as to prevent heap overflows. + + "array" uses arrays in the JVM heap + + "bytebuffer" uses off-heap memory within the JVM. + + Both "array" and "bytebuffer" will consume memory proportional + to the partition size * active+pending block uploads. + + To avoid running out of memory, keep the fs.s3a.threads.max and + fs.s3a.max.total.tasks values low. + No effect if fs.s3a.block.upload is false. + + + fs.s3a.multipart.size + 104857600 + + How big (in bytes) to split upload or copy operations up into. + + + ``` @@ -992,7 +1020,7 @@ will block until the upload is completed. ##### Disk upload `fs.s3a.block.output.buffer=disk` -When `fs.s3a.block.output.buffer` is set to `array`, all data is buffered +When `fs.s3a.block.output.buffer` is set to `disk`, all data is buffered to local hard disks in 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 @@ -1012,24 +1040,70 @@ throughput. Instead the bandwidth is shared between all threads, slowing them all down. We recommend a low number of threads, especially when uploading over long-haul connections. +Because the upload data is buffered via disk, there is little risk of heap +overflow as the queue of pending uploads (limited by `fs.s3a.max.total.tasks`) +increases. + + ```xml + + fs.s3a.block.upload + true + + + + fs.s3a.block.output.buffer + disk + + fs.s3a.threads.max 5 - Maximum number of concurrent active (part)uploads, - which each use a thread from the threadpool. + + + + fs.s3a.max.total.tasks + 20 ``` -Because the upload data is buffered via disk, there is little risk of heap -overflow as the queue of pending uploads (limited by `fs.s3a.max.total.tasks`) -increases. + + +##### Block Upload with ByteBuffers: `fs.s3a.block.output.buffer=bytebuffer` + +When `fs.s3a.block.output.buffer` is set to `bytebuffer`, all data is buffered +in "Direct" ByteBuffers prior to upload. This *may* be faster than buffering to disk. + +These buffers are in the memory of the JVM, albeit not in the Java Heap itself. +The amount of data which can be buffered is +limited by the Jva 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. + +To reduce the risk of this, the number of +threads performing uploads and the size of queued upload operations must be +kept low. ```xml + + fs.s3a.block.upload + true + + + + fs.s3a.block.output.buffer + array + + + + fs.s3a.threads.max + 3 + + fs.s3a.max.total.tasks - 20 - Number of (part)uploads allowed to the queue before - blocking additional uploads. + 1 ``` @@ -1037,29 +1111,38 @@ increases. ##### Array upload: `fs.s3a.block.output.buffer=array` When `fs.s3a.block.output.buffer` is set to `array`, all data is buffered -in memory prior to upload. This *may* be faster than buffering to disk. +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` -This `array` option is similar to the in-memory-only stream implemented in -`S3AFastOutputStream`: the amount of data which can be buffered is -limited by available heap. The slower the write bandwidth to S3, the greater +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. To reduce the risk of this, the number of threads performing uploads and the size of queued upload operations must be kept low. As an example: three threads and one queued task will consume `(3 + 1) * fs.s3a.multipart.size` bytes. ```xml + + fs.s3a.block.upload + true + + + + fs.s3a.block.output.buffer + array + + fs.s3a.threads.max 3 - Maximum number of concurrent active (part)uploads, - which each use a thread from the threadpool. fs.s3a.max.total.tasks 1 - Number of (part)uploads allowed to the queue before - blocking additional uploads. ``` @@ -1551,6 +1634,45 @@ Set `fs.s3a.connection.maximum` to a larger value (and at least as large as ### Out of heap memory when writing to S3A +This can happen when using the block output stream (`fs.s3a.block.output=true`) +and in-memory buffering (either `fs.s3a.block.output.buffer=array` or +`fs.s3a.block.output.buffer=bytebuffer`). + +More data is being generated than in the JVM than it can upload to S3 —and +so much data has been buffered that the JVM has run out of memory. + +Fixes + +1. Increase heap capacity. This is a short term measure —the problem may +return. +1. Decrease the values of `fs.s3a.threads.max` and `fs.s3a.max.total.tasks` + so that less uploads can be pending. You may need to decrease the value +of `fs.s3a.multipart.size` to reduce the amount of memory buffered in each +queued operation. +1. Switch to disk buffering: `fs.s3a.block.output.buffer=disk`. The upper limit +on buffered data becomes limited to that of disk capacity. + +If using disk buffering, the number of total tasks can be kept high, as +pending uploads consume little memory. Keeping the thread count low can +reduce bandwidth requirements on the uploads + +## When writing to S3A: "java.io.FileNotFoundException: Completing multi-part upload" + + +``` +java.io.FileNotFoundException: Completing multi-part upload on fork-5/test/multipart/1c397ca6-9dfb-4ac1-9cf7-db666673246b: com.amazonaws.services.s3.model.AmazonS3Exception: The specified upload does not exist. The upload ID may be invalid, or the upload may have been aborted or completed. (Service: Amazon S3; Status Code: 404; Error Code: NoSuchUpload; Request ID: 84FF8057174D9369), S3 Extended Request ID: Ij5Yn6Eq/qIERH4Z6Io3YL2t9/qNZ7z9gjPb1FrTtTovZ8k1MXqh+zCYYjqmfJ/fCY6E1+JR9jA= + at com.amazonaws.http.AmazonHttpClient.handleErrorResponse(AmazonHttpClient.java:1182) + at com.amazonaws.http.AmazonHttpClient.executeOneRequest(AmazonHttpClient.java:770) + 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.S3ABlockOutputStream$MultiPartUpload.complete(S3ABlockOutputStream.java:473) + at org.apache.hadoop.fs.s3a.S3ABlockOutputStream$MultiPartUpload.access$200(S3ABlockOutputStream.java:382) + at org.apache.hadoop.fs.s3a.S3ABlockOutputStream.close(S3ABlockOutputStream.java:272) + at org.apache.hadoop.fs.FSDataOutputStream$PositionCache.close(FSDataOutputStream.java:72) + at org.apache.hadoop.fs.FSDataOutputStream.close(FSDataOutputStream.java:106) +`` ## Visible S3 Inconsistency diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractDistCp.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractDistCp.java index 28278feea9057..8ba1acf097ffb 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractDistCp.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractDistCp.java @@ -18,24 +18,26 @@ package org.apache.hadoop.fs.contract.s3a; -import static org.apache.hadoop.fs.s3a.Constants.MIN_MULTIPART_THRESHOLD; -import static org.apache.hadoop.fs.s3a.Constants.MULTIPART_SIZE; +import static org.apache.hadoop.fs.s3a.Constants.*; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.tools.contract.AbstractContractDistCpTest; /** * Contract test suite covering S3A integration with DistCp. + * Uses the block output stream, buffered to disk. This is the + * recommended output mechanism for DistCP due to its scalability. */ public class ITestS3AContractDistCp extends AbstractContractDistCpTest { - private static final long MULTIPART_SETTING = 8 * 1024 * 1024; // 8 MB + private static final long MULTIPART_SETTING = MULTIPART_MIN_SIZE; @Override protected Configuration createConfiguration() { Configuration newConf = super.createConfiguration(); - newConf.setLong(MIN_MULTIPART_THRESHOLD, MULTIPART_SETTING); newConf.setLong(MULTIPART_SIZE, MULTIPART_SETTING); + newConf.setBoolean(BLOCK_OUTPUT, true); + newConf.set(BLOCK_OUTPUT_BUFFER, BLOCK_OUTPUT_BUFFER_DISK); return newConf; } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ABlockingThreadPool.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ABlockingThreadPool.java index 4444d0c9769ca..c6ba52f1410d7 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ABlockingThreadPool.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ABlockingThreadPool.java @@ -51,7 +51,7 @@ public void setUp() throws Exception { conf = new Configuration(); conf.setLong(Constants.MIN_MULTIPART_THRESHOLD, 5 * 1024 * 1024); conf.setLong(Constants.MULTIPART_SIZE, 5 * 1024 * 1024); - conf.setInt(Constants.MAX_THREADS, 2); + conf.setInt(Constants.MAX_THREADS, 1); conf.setInt(Constants.MAX_TOTAL_TASKS, 1); } @@ -71,7 +71,9 @@ public void testRegularMultiPartUpload() throws Exception { @Test public void testFastMultiPartUpload() throws Exception { - conf.setBoolean(Constants.FAST_UPLOAD, true); + conf.setBoolean(Constants.BLOCK_OUTPUT, true); + conf.set(Constants.BLOCK_OUTPUT_BUFFER, + Constants.BLOCK_OUTPUT_BYTEBUFFER); fs = S3ATestUtils.createTestFileSystem(conf); ContractTestUtils.createAndVerifyFile(fs, getTestPath(), 16 * 1024 * 1024); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEncryptionFastOutputStream.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEncryptionBlockOutputStream.java similarity index 78% rename from hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEncryptionFastOutputStream.java rename to hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEncryptionBlockOutputStream.java index c06fed1a8f2db..fe599a85a2d22 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEncryptionFastOutputStream.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEncryptionBlockOutputStream.java @@ -21,15 +21,16 @@ import org.apache.hadoop.conf.Configuration; /** - * Run the encryption tests against the Fast output stream. - * This verifies that both file writing paths can encrypt their data. + * Run the encryption tests against the block output stream. */ -public class ITestS3AEncryptionFastOutputStream extends ITestS3AEncryption { +public class ITestS3AEncryptionBlockOutputStream extends ITestS3AEncryption { @Override protected Configuration createConfiguration() { Configuration conf = super.createConfiguration(); - conf.setBoolean(Constants.FAST_UPLOAD, true); + conf.setBoolean(Constants.BLOCK_OUTPUT, true); + conf.set(Constants.BLOCK_OUTPUT_BUFFER, + Constants.BLOCK_OUTPUT_BYTEBUFFER); return conf; } } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java index 5b4fe9e5d6883..4f793aa18b0d2 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java @@ -87,8 +87,12 @@ public static S3AFileSystem createTestFileSystem(Configuration conf, } S3AFileSystem fs1 = new S3AFileSystem(); //enable purging in tests - conf.setBoolean(PURGE_EXISTING_MULTIPART, true); - conf.setInt(PURGE_EXISTING_MULTIPART_AGE, 0); + if (purge) { + conf.setBoolean(PURGE_EXISTING_MULTIPART, true); + // but a long delay so that parallel multipart tests don't + // suddenly start timing out + conf.setInt(PURGE_EXISTING_MULTIPART_AGE, 30 * 60); + } fs1.initialize(testURI, conf); return fs1; } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestDataBlocks.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestDataBlocks.java new file mode 100644 index 0000000000000..42801b0d87b33 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestDataBlocks.java @@ -0,0 +1,123 @@ +/* + * 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 org.apache.hadoop.fs.contract.ContractTestUtils; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.Timeout; + +/** + * Unit tests for {@link S3ADataBlocks}. + */ +public class TestDataBlocks extends Assert { + + @Rule + public Timeout testTimeout = new Timeout(30 * 1000); + + @Before + public void nameThread() { + Thread.currentThread().setName("JUnit"); + } + + /** + * Test the {@link S3ADataBlocks.ByteBufferBlockFactory}. + * That code implements an input stream over a ByteBuffer, and has to + * return the buffer to the pool after the read complete. + * + * This test verifies the basic contract of the process. + */ + @Test + public void testByteBufferIO() throws Throwable { + try(S3ADataBlocks.ByteBufferBlockFactory factory = + new S3ADataBlocks.ByteBufferBlockFactory()) { + int limit = 128; + S3ADataBlocks.ByteBufferBlockFactory.ByteBufferBlock block + = factory.create(limit); + assertEquals("outstanding buffers in " + factory, + 1, factory.getOutstandingBufferCount()); + + byte[] buffer = ContractTestUtils.toAsciiByteArray("test data"); + int bufferLen = buffer.length; + block.write(buffer, 0, bufferLen); + assertEquals(bufferLen, block.dataSize()); + assertEquals("capacity in " + block, + limit - bufferLen, block.remainingCapacity()); + assertTrue("hasCapacity(64) in " + block, block.hasCapacity(64)); + assertTrue("No capacity in " + block, + block.hasCapacity(limit - bufferLen)); + + // now start the write + S3ADataBlocks.ByteBufferBlockFactory.ByteBufferInputStream + stream = block.openForUpload(); + assertTrue("!hasRemaining() in " + stream, stream.hasRemaining()); + int expected = bufferLen; + assertEquals("wrong available() in " + stream, + expected, stream.available()); + + assertEquals('t', stream.read()); + expected --; + assertEquals("wrong available() in " + stream, + expected, stream.available()); + + // close the block. The buffer must remain outstanding here; + // the stream manages the lifecycle of it now + block.close(); + assertEquals("outstanding buffers in " + factory, + 1, factory.getOutstandingBufferCount()); + block.close(); + + // read into a byte array with an offset + int offset = 5; + byte[] in = new byte[limit]; + assertEquals(2, stream.read(in, offset, 2)); + assertEquals('e', in[offset]); + assertEquals('s', in[offset + 1]); + expected -= 2; + assertEquals("wrong available() in " + stream, + expected, stream.available()); + + // read to end + byte[] remainder = new byte[limit]; + int c; + int index = 0; + while ((c = stream.read()) >= 0) { + remainder[index++] = (byte) c; + } + assertEquals(expected, index); + assertEquals('a', remainder[--index]); + + assertEquals("wrong available() in " + stream, + 0, stream.available()); + assertTrue("hasRemaining() in " + stream, !stream.hasRemaining()); + + // when the stream is closed, the data should be returned + stream.close(); + assertEquals("outstanding buffers in " + factory, + 0, factory.getOutstandingBufferCount()); + stream.close(); + assertEquals("outstanding buffers in " + factory, + 0, factory.getOutstandingBufferCount()); + + } + + } +} diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/AbstractSTestS3AHugeFiles.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/AbstractSTestS3AHugeFiles.java index d06ad678f4676..eab80ba3c642c 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/AbstractSTestS3AHugeFiles.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/AbstractSTestS3AHugeFiles.java @@ -32,12 +32,10 @@ import org.junit.Assume; import org.junit.FixMethodOrder; import org.junit.Test; -import org.junit.internal.AssumptionViolatedException; import org.junit.runners.MethodSorters; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.FileNotFoundException; import java.io.IOException; import static org.apache.hadoop.fs.contract.ContractTestUtils.toHuman; @@ -83,7 +81,6 @@ protected Configuration createConfiguration() { Configuration configuration = super.createConfiguration(); configuration.setLong(SOCKET_SEND_BUFFER, BLOCKSIZE); configuration.setLong(SOCKET_RECV_BUFFER, BLOCKSIZE); - configuration.setLong(MIN_MULTIPART_THRESHOLD, 5 * _1MB); configuration.setLong(MIN_MULTIPART_THRESHOLD, MULTIPART_MIN_SIZE); configuration.setInt(MULTIPART_SIZE, MULTIPART_MIN_SIZE); configuration.set(USER_AGENT_PREFIX, "STestS3AHugeFileCreate"); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/STestS3AHugeDistCP.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/STestS3AHugeDistCP.java deleted file mode 100644 index 75c1794cabf89..0000000000000 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/STestS3AHugeDistCP.java +++ /dev/null @@ -1,32 +0,0 @@ -/* - * 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.scale; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * Test very large scale distCP upload using the huge file options. - */ -public class STestS3AHugeDistCP extends S3AScaleTestBase { - private static final Logger LOG = LoggerFactory.getLogger( - STestS3AHugeDistCP.class); - - -} diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/STestS3AHugeFilesFastUpload.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/STestS3AHugeFilesByteArrayBlocks.java similarity index 71% rename from hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/STestS3AHugeFilesFastUpload.java rename to hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/STestS3AHugeFilesByteArrayBlocks.java index 7e6da5791ffb2..4348b93e6bf40 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/STestS3AHugeFilesFastUpload.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/STestS3AHugeFilesByteArrayBlocks.java @@ -20,15 +20,18 @@ import org.apache.hadoop.conf.Configuration; -import static org.apache.hadoop.fs.s3a.Constants.FAST_UPLOAD; -import static org.apache.hadoop.fs.s3a.Constants.MIN_MULTIPART_THRESHOLD; +import static org.apache.hadoop.fs.s3a.Constants.BLOCK_OUTPUT; +import static org.apache.hadoop.fs.s3a.Constants.BLOCK_OUTPUT_BUFFER; +import static org.apache.hadoop.fs.s3a.Constants.BLOCK_OUTPUT_BYTEBUFFER; -public class STestS3AHugeFilesFastUpload extends AbstractSTestS3AHugeFiles { +public class STestS3AHugeFilesByteArrayBlocks + extends AbstractSTestS3AHugeFiles { @Override protected Configuration createConfiguration() { final Configuration configuration = super.createConfiguration(); - configuration.setBoolean(FAST_UPLOAD, true); + configuration.setBoolean(BLOCK_OUTPUT, true); + configuration.set(BLOCK_OUTPUT_BUFFER, BLOCK_OUTPUT_BYTEBUFFER); return configuration; } } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/STestS3AHugeFilesDiskBlocks.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/STestS3AHugeFilesDiskBlocks.java index 9731102e7e95b..3f0449dcb9878 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/STestS3AHugeFilesDiskBlocks.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/STestS3AHugeFilesDiskBlocks.java @@ -23,10 +23,6 @@ import static org.apache.hadoop.fs.s3a.Constants.BLOCK_OUTPUT; import static org.apache.hadoop.fs.s3a.Constants.BLOCK_OUTPUT_BUFFER; import static org.apache.hadoop.fs.s3a.Constants.BLOCK_OUTPUT_BUFFER_DISK; -import static org.apache.hadoop.fs.s3a.Constants.FAST_UPLOAD; -import static org.apache.hadoop.fs.s3a.Constants.MIN_MULTIPART_THRESHOLD; -import static org.apache.hadoop.fs.s3a.Constants.MULTIPART_MIN_SIZE; -import static org.apache.hadoop.fs.s3a.Constants.MULTIPART_SIZE; public class STestS3AHugeFilesDiskBlocks extends AbstractSTestS3AHugeFiles { diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/STestS3AHugeFilesMemoryBlocks.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/STestS3AHugeFilesMemoryBlocks.java index c3a930fbd6e7f..0d390e7886847 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/STestS3AHugeFilesMemoryBlocks.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/STestS3AHugeFilesMemoryBlocks.java @@ -23,7 +23,6 @@ import static org.apache.hadoop.fs.s3a.Constants.BLOCK_OUTPUT; import static org.apache.hadoop.fs.s3a.Constants.BLOCK_OUTPUT_BUFFER; import static org.apache.hadoop.fs.s3a.Constants.BLOCK_OUTPUT_BUFFER_ARRAY; -import static org.apache.hadoop.fs.s3a.Constants.BLOCK_OUTPUT_BUFFER_DISK; public class STestS3AHugeFilesMemoryBlocks extends AbstractSTestS3AHugeFiles { From c342febaf25fb084f54b744ef2eb590a1d3d098e Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Wed, 21 Sep 2016 23:16:12 +0100 Subject: [PATCH 20/20] Patch 004 * more debug statements * 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 * new Progress log for logging progress @ debug level in s3a. Why? Because logging events every 8KB gets too chatty when debugging many-MB uploads. * gauges of active block uploads wired up. --- .../src/main/resources/core-default.xml | 6 +- hadoop-tools/hadoop-aws/pom.xml | 4 +- .../hadoop/fs/s3a/S3ABlockOutputStream.java | 91 +++-- .../apache/hadoop/fs/s3a/S3ADataBlocks.java | 354 +++++++++++------- .../apache/hadoop/fs/s3a/S3AFileSystem.java | 17 +- .../hadoop/fs/s3a/S3AInstrumentation.java | 3 +- .../site/markdown/tools/hadoop-aws/index.md | 25 +- .../apache/hadoop/fs/s3a/TestDataBlocks.java | 4 +- 8 files changed, 295 insertions(+), 209 deletions(-) diff --git a/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml b/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml index 95e9d61b28bb0..f160c29dba618 100644 --- a/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml +++ b/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml @@ -1101,7 +1101,7 @@ - fs.s3a.block.upload + fs.s3a.block.output false Use the incremental block upload mechanism with @@ -1118,7 +1118,7 @@ disk Buffering mechanism to use when using S3A Block upload - (fs.s3a.block.upload=true). Values: disk, array, bytebuffer. + (fs.s3a.block.output=true). Values: disk, array, bytebuffer. "disk" will use the directories listed in fs.s3a.buffer.dir as the location(s) to save data prior to being uploaded. @@ -1133,7 +1133,7 @@ To avoid running out of memory, keep the fs.s3a.threads.max and fs.s3a.max.total.tasks values low. - No effect if fs.s3a.block.upload is false. + No effect if fs.s3a.block.output is false. diff --git a/hadoop-tools/hadoop-aws/pom.xml b/hadoop-tools/hadoop-aws/pom.xml index ce5b214830546..19f1aa9f68c97 100644 --- a/hadoop-tools/hadoop-aws/pom.xml +++ b/hadoop-tools/hadoop-aws/pom.xml @@ -37,7 +37,7 @@ ${project.build.directory}/test - 10 + 5 1800 @@ -147,7 +147,6 @@ fork-${surefire.forkNumber} - ${fs.s3a.scale.test.huge.filesize} ${fs.s3a.scale.test.timeout} @@ -188,7 +187,6 @@ ${fs.s3a.scale.test.huge.filesize} ${fs.s3a.scale.test.timeout} - diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java index 2212d3a91e6b1..b31600171af76 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java @@ -1,4 +1,4 @@ -/** +/* * 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 @@ -55,6 +55,7 @@ import java.util.concurrent.Callable; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; import static org.apache.hadoop.fs.s3a.S3AUtils.*; import static org.apache.hadoop.fs.s3a.Statistic.*; @@ -93,7 +94,7 @@ class S3ABlockOutputStream extends OutputStream { private final ProgressListener progressListener; private final ListeningExecutorService executorService; - /** Retry policy for multipart commits; not all AWS SDK versions retry */ + /** Retry policy for multipart commits; not all AWS SDK versions retry that. */ private final RetryPolicy retryPolicy = RetryPolicies.retryUpToMaximumCountWithProportionalSleep( 5, @@ -102,15 +103,15 @@ class S3ABlockOutputStream extends OutputStream { /** * Factory for blocks. */ - private final S3ADataBlocks.AbstractBlockFactory blockFactory; + 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 */ + /** Multipart upload details; null means none started. */ private MultiPartUpload multiPartUpload; /** Closed flag. */ - private volatile boolean closed; + private final AtomicBoolean closed = new AtomicBoolean(false); /** Current data block. Null means none currently active */ private S3ADataBlocks.DataBlock currentBlock; @@ -123,7 +124,7 @@ class S3ABlockOutputStream extends OutputStream { /** * An S3A output stream which uploads partitions in a separate pool of - * threads; different {@link S3ADataBlocks.AbstractBlockFactory} + * threads; different {@link S3ADataBlocks.BlockFactory} * instances can control where data is buffered. * * @param fs S3AFilesystem @@ -140,7 +141,7 @@ class S3ABlockOutputStream extends OutputStream { String key, Progressable progress, long blockSize, - S3ADataBlocks.AbstractBlockFactory blockFactory, + S3ADataBlocks.BlockFactory blockFactory, S3AInstrumentation.OutputStreamStatistics statistics) throws IOException { this.fs = fs; @@ -187,8 +188,8 @@ private synchronized void maybeCreateDestStream() throws IOException { * @throws IOException if the filesystem is closed. */ void checkOpen() throws IOException { - if (closed) { - throw new IOException("Filesystem closed"); + if (closed.get()) { + throw new IOException("Filesystem " + bucket + " closed"); } } @@ -266,11 +267,12 @@ private synchronized void uploadCurrentBlock() throws IOException { if (multiPartUpload == null) { multiPartUpload = initiateMultiPartUpload(); } - multiPartUpload.uploadBlockAsync(currentBlock); - // close the block - currentBlock.close(); - // set it to null, so the next write will create a new block. - currentBlock = null; + try { + multiPartUpload.uploadBlockAsync(currentBlock); + } finally { + // set the block to null, so the next write will create a new block. + currentBlock = null; + } } /** @@ -284,18 +286,19 @@ private synchronized void uploadCurrentBlock() throws IOException { */ @Override public synchronized void close() throws IOException { - if (closed) { + if (closed.getAndSet(true)) { + // already closed + LOG.debug("Ignoring close() as stream is already closed"); return; } - closed = true; LOG.debug("{}: Closing block #{}: current block= {}, data to upload = {}", this, blockCount, currentBlock == null ? "(none)" : currentBlock, - currentBlock == null ? 0 : currentBlock.dataSize() ); + currentBlock == null ? 0 : currentBlock.dataSize()); try { if (multiPartUpload == null) { - if (currentBlock != null){ + if (currentBlock != null) { // no uploads of data have taken place, put the single block up. putObject(); } @@ -304,7 +307,7 @@ public synchronized void close() throws IOException { // put up the current then wait if (currentBlock != null && currentBlock.hasData()) { //send last part - multiPartUpload.uploadBlockAsync(currentBlock); + uploadCurrentBlock(); } // wait for the partial uploads to finish final List partETags = @@ -365,7 +368,7 @@ private void putObject() throws IOException { final PutObjectRequest putObjectRequest = fs.newPutObjectRequest(key, om, - block.openForUpload()); + block.startUpload()); long transferQueueTime = now(); BlockUploadProgress callback = new BlockUploadProgress( @@ -435,6 +438,7 @@ public MultiPartUpload(String uploadId) { /** * Upload a block of data. + * This will take the block * @param block block to upload * @throws IOException upload failure */ @@ -442,7 +446,7 @@ private void uploadBlockAsync(final S3ADataBlocks.DataBlock block) throws IOException { LOG.debug("Queueing upload of {}", block); final int size = block.dataSize(); - final InputStream uploadStream = block.openForUpload(); + final InputStream uploadStream = block.startUpload(); final int currentPartNumber = partETagsFutures.size() + 1; final UploadPartRequest request = new UploadPartRequest() @@ -517,11 +521,13 @@ private CompleteMultipartUploadResult complete(List partETags) throws IOException { int retryCount = 0; AmazonClientException lastException; + String operation = + String.format("Completing multi-part upload for key '%s'," + + " id '%s' with %s partitions ", + key, uploadId, partETags.size()); do { try { - LOG.debug("Completing multi-part upload for key '{}'," + - " id '{}' with {} partitions " , - key, uploadId, partETags.size()); + LOG.debug(operation); return getS3Client().completeMultipartUpload( new CompleteMultipartUploadRequest(bucket, key, @@ -531,11 +537,10 @@ private CompleteMultipartUploadResult complete(List partETags) lastException = e; statistics.exceptionInMultipartComplete(); } - } while (shouldRetry(lastException, retryCount++)); + } while (shouldRetry(operation, lastException, retryCount++)); // this point is only reached if the operation failed more than // the allowed retry count - throw translateException("Completing multi-part upload", key, - lastException); + throw translateException(operation, key, lastException); } /** @@ -545,18 +550,20 @@ public void abort() { int retryCount = 0; AmazonClientException lastException; fs.incrementStatistic(OBJECT_MULTIPART_UPLOAD_ABORTED); + String operation = + String.format("Aborting multi-part upload for key '%s', id '%s", + key, uploadId); do { try { - LOG.debug("Aborting multi-part upload for key '{}', id '{}'", - key, uploadId); - getS3Client().abortMultipartUpload(new AbortMultipartUploadRequest(bucket, - key, uploadId)); + LOG.debug(operation); + getS3Client().abortMultipartUpload( + new AbortMultipartUploadRequest(bucket, key, uploadId)); return; } catch (AmazonClientException e) { lastException = e; statistics.exceptionInMultipartAbort(); } - } while (shouldRetry(lastException, retryCount++)); + } while (shouldRetry(operation, lastException, retryCount++)); // this point is only reached if the operation failed more than // the allowed retry count LOG.warn("Unable to abort multipart upload, you may need to purge " + @@ -568,19 +575,31 @@ public void abort() { * be attempted again. * If a retry is advised, the exception is automatically logged and * the filesystem statistic {@link Statistic#IGNORED_ERRORS} incremented. + * The method then sleeps for the sleep time suggested by the sleep policy; + * if the sleep is interrupted then {@code Thread.interrupted()} is set + * to indicate the thread was interrupted; then false is returned. + * + * @param operation operation for log message * @param e exception raised. * @param retryCount number of retries already attempted * @return true if another attempt should be made */ - private boolean shouldRetry(AmazonClientException e, int retryCount) { + private boolean shouldRetry(String operation, + AmazonClientException e, + int retryCount) { try { - boolean retry = retryPolicy.shouldRetry(e, retryCount, 0, true) - == RetryPolicy.RetryAction.RETRY; + RetryPolicy.RetryAction retryAction = + retryPolicy.shouldRetry(e, retryCount, 0, true); + boolean retry = retryAction == RetryPolicy.RetryAction.RETRY; if (retry) { fs.incrementStatistic(IGNORED_ERRORS); - LOG.info("Retrying operation after exception " + e, e); + LOG.info("Retrying {} after exception ", operation, e); + Thread.sleep(retryAction.delayMillis); } return retry; + } catch (InterruptedException ex) { + Thread.interrupted(); + return false; } catch (Exception ignored) { return false; } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ADataBlocks.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ADataBlocks.java index 15e8c2d73c5ed..384db6ebef492 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ADataBlocks.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ADataBlocks.java @@ -37,9 +37,12 @@ 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 static org.apache.hadoop.fs.s3a.S3ADataBlocks.DataBlock.DestState.*; +import static org.apache.hadoop.fs.s3a.S3ADataBlocks.DataBlock.DestState.Closed; +import static org.apache.hadoop.fs.s3a.S3ADataBlocks.DataBlock.DestState.Upload; +import static org.apache.hadoop.fs.s3a.S3ADataBlocks.DataBlock.DestState.Writing; /** * Set of classes to support output streaming into blocks which are then @@ -48,16 +51,58 @@ class S3ADataBlocks { static final Logger LOG = LoggerFactory.getLogger(S3ADataBlocks.class); - static abstract class AbstractBlockFactory implements Closeable { + /** + * Validate args to a write command. + * @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 + '"'); + } + } - protected S3AFileSystem owner; + /** + * Base class for block factories. + */ + static abstract class BlockFactory implements Closeable { /** - * Bind to the factory owner. - * @param fs owner filesystem + * Owner. */ - void init(S3AFileSystem fs) { - this.owner = fs; + protected final S3AFileSystem owner; + + protected BlockFactory(S3AFileSystem owner) { + this.owner = owner; } /** @@ -77,51 +122,17 @@ public void close() throws IOException { } } - static class MemoryBlockFactory extends AbstractBlockFactory { - - @Override - void init(S3AFileSystem fs) { - super.init(fs); - } - - @Override - DataBlock create(int limit) throws IOException { - return new ByteArrayBlock(limit); - } - - - } - - /** - * Buffer blocks to disk. - */ - static class DiskBlockFactory extends AbstractBlockFactory { - - @Override - void init(S3AFileSystem fs) { - super.init(fs); - } - - @Override - DataBlock create(int limit) throws IOException { - File destFile = owner.getDirectoryAllocator() - .createTmpFileForWrite("s3ablock", limit, owner.getConf()); - return new FileBlock(destFile, limit); - } - } - /** * This represents a block being uploaded. */ static abstract class DataBlock implements Closeable { - enum DestState {Writing, Upload, Closed} - private volatile DestState state = Writing; - protected synchronized void enterState(DestState current, DestState next) { + protected synchronized void enterState(DestState current, DestState next) + throws IOException { verifyState(current); - LOG.debug("{}: entering state {}" , this, next); + LOG.debug("{}: entering state {}", this, next); state = next; } @@ -130,10 +141,11 @@ protected synchronized void enterState(DestState current, DestState next) { * @param expected expected state. * @throws IllegalStateException if the DataBlock is in the wrong state */ - protected void verifyState(DestState expected) { - Preconditions.checkState(state == expected, - "Expected stream state " + expected + " -but actual state is " + state - + " in " + this); + protected void verifyState(DestState expected) throws IOException { + if (expected != null && state != expected) { + throw new IOException("Expected stream state " + expected + + " -but actual state is " + state + " in " + this); + } } DestState getState() { @@ -179,7 +191,7 @@ boolean hasData() { * @return number of bytes written * @throws IOException trouble */ - int write(byte buffer[], int offset, int length) throws IOException { + 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"); @@ -207,7 +219,8 @@ void flush() throws IOException { * @return the stream * @throws IOException trouble */ - InputStream openForUpload() throws IOException { + InputStream startUpload() throws IOException { + LOG.debug("Start datablock upload"); enterState(Writing, Upload); return null; } @@ -219,12 +232,51 @@ InputStream openForUpload() throws IOException { */ protected synchronized boolean enterClosedState() { if (!state.equals(Closed)) { - enterState(state, Closed); + try { + enterState(null, Closed); + } catch (IOException 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); + } + } /** @@ -238,7 +290,7 @@ protected synchronized boolean enterClosedState() { static class ByteArrayBlock extends DataBlock { private ByteArrayOutputStream buffer; - private int limit; + private final int limit; // cache data size so that it is consistent after the buffer is reset. private Integer dataSize; @@ -257,8 +309,8 @@ int dataSize() { } @Override - InputStream openForUpload() throws IOException { - super.openForUpload(); + InputStream startUpload() throws IOException { + super.startUpload(); dataSize = buffer.size(); ByteArrayInputStream bufferData = new ByteArrayInputStream( buffer.toByteArray()); @@ -286,30 +338,43 @@ int write(byte[] b, int offset, int len) throws IOException { } @Override - public void close() throws IOException { - if (enterClosedState()) { - LOG.debug("Closed {}", this); - buffer = null; - } + protected void innerClose() { + buffer = null; + } + + @Override + public String toString() { + return "ByteArrayBlock{" + + "state=" + getState() + + ", buffer=" + buffer + + ", 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 FileBlock} buffer delegates the work of deleting files to - * the {@link FileBlock.FileDeletingInputStream}. Here 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 AbstractBlockFactory { + static class ByteBufferBlockFactory extends BlockFactory { - private final DirectBufferPool bufferPool =new DirectBufferPool(); + 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); @@ -337,11 +402,9 @@ public int getOutstandingBufferCount() { @Override public String toString() { - final StringBuilder sb = new StringBuilder( - "ByteBufferBlockFactory{"); - sb.append(", buffersOutstanding=").append(buffersOutstanding.toString()); - sb.append('}'); - return sb.toString(); + return "ByteBufferBlockFactory{" + + "buffersOutstanding=" + buffersOutstanding + + '}'; } /** @@ -373,8 +436,8 @@ int dataSize() { } @Override - ByteBufferInputStream openForUpload() throws IOException { - super.openForUpload(); + ByteBufferInputStream startUpload() throws IOException { + super.startUpload(); dataSize = bufferCapacityUsed(); // set the buffer up from reading from the beginning buffer.limit(buffer.position()); @@ -405,23 +468,18 @@ int write(byte[] b, int offset, int len) throws IOException { } @Override - public void close() throws IOException { - if (enterClosedState()) { - LOG.debug("Closed {}", this); - buffer = null; - } + protected void innerClose() { + buffer = null; } @Override public String toString() { - final StringBuilder sb = new StringBuilder( - "ByteBufferBlock{"); - sb.append("state=").append(getState()); - sb.append(", dataSize=").append(dataSize()); - sb.append(", limit=").append(bufferSize); - sb.append(", remainingCapacity=").append(remainingCapacity()); - sb.append('}'); - return sb.toString(); + return "ByteBufferBlock{" + + "state=" + getState() + + ", dataSize=" + dataSize() + + ", limit=" + bufferSize + + ", remainingCapacity=" + remainingCapacity() + + '}'; } } @@ -443,8 +501,11 @@ class ByteBufferInputStream extends InputStream { this.byteBuffer = byteBuffer; } + /** + * Return the buffer to the pool after the stream is closed. + */ @Override - public synchronized void close() throws IOException { + public synchronized void close() { if (byteBuffer != null) { LOG.debug("releasing buffer"); releaseBuffer(byteBuffer); @@ -571,21 +632,44 @@ public String 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.getDirectoryAllocator() + .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 FileBlock extends DataBlock { + static class DiskBlock extends DataBlock { + protected int bytesWritten; private File bufferFile; - private int limit; - protected int bytesWritten; - private BufferedOutputStream out; private InputStream uploadStream; - public FileBlock(File bufferFile, int limit) + DiskBlock(File bufferFile, int limit) throws FileNotFoundException { this.limit = limit; this.bufferFile = bufferFile; @@ -618,10 +702,11 @@ int write(byte[] b, int offset, int len) throws IOException { } @Override - InputStream openForUpload() throws IOException { - super.openForUpload(); + InputStream startUpload() throws IOException { + super.startUpload(); try { out.flush(); + out.close(); } finally { out.close(); out = null; @@ -630,20 +715,27 @@ InputStream openForUpload() throws IOException { return new FileDeletingInputStream(uploadStream); } + /** + * The close operation will delete the destination file if it still + * exists. + * @throws IOException IO problems + */ @Override - public synchronized void close() throws IOException { + protected void innerClose() throws IOException { final DestState state = getState(); LOG.debug("Closing {}", this); - enterClosedState(); - final boolean bufferExists = bufferFile.exists(); switch (state) { case Writing: - if (bufferExists) { + if (bufferFile.exists()) { // file was not uploaded LOG.debug("Deleting buffer file as upload did not start"); - bufferFile.delete(); + 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; @@ -653,6 +745,10 @@ public synchronized void close() throws IOException { } } + /** + * Flush operation will flush to disk. + * @throws IOException IOE raised on FileOutputStream + */ @Override void flush() throws IOException { verifyState(Writing); @@ -661,35 +757,40 @@ void flush() throws IOException { @Override public String toString() { - final StringBuilder sb = new StringBuilder( - "FileBlock{"); - sb.append("destFile=").append(bufferFile); - sb.append(", state=").append(getState()); - sb.append(", dataSize=").append(dataSize()); - sb.append(", limit=").append(limit); - sb.append('}'); - return sb.toString(); + String sb = "FileBlock{" + "destFile=" + bufferFile + + ", state=" + getState() + + ", dataSize=" + dataSize() + + ", limit=" + limit + + '}'; + return sb; } /** * An input stream which deletes the buffer file when closed. */ private class FileDeletingInputStream extends ForwardingInputStream { + private final AtomicBoolean closed = new AtomicBoolean(false); FileDeletingInputStream(InputStream source) { super(source); } + /** + * Delete the input file when closed + * @throws IOException IO problem + */ @Override public void close() throws IOException { super.close(); - bufferFile.delete(); + if (!closed.getAndSet(true)) { + bufferFile.delete(); + } } } } /** - * Stream which forwards everything to its inner class. + * Stream which forwards everything to another stream. * For ease of subclassing. */ @SuppressWarnings({ @@ -700,7 +801,7 @@ static class ForwardingInputStream extends InputStream { protected final InputStream source; - public ForwardingInputStream(InputStream source) { + ForwardingInputStream(InputStream source) { this.source = source; } @@ -725,7 +826,7 @@ public int available() throws IOException { } public void close() throws IOException { - LOG.debug("Closing inner stream"); + LOG.debug("Closing source stream"); source.close(); } @@ -742,41 +843,4 @@ public boolean markSupported() { } } - - /** - * Validate args to write command. - * @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(); - } - } - - /** - * Create a factory. - * @param name factory name -the option from {@link Constants}. - * @return the factory, ready to be initialized. - * @throws IllegalArgumentException if the name is unknown. - */ - static AbstractBlockFactory createFactory(String name) { - switch (name) { - case Constants.BLOCK_OUTPUT_BUFFER_ARRAY: - return new MemoryBlockFactory(); - case Constants.BLOCK_OUTPUT_BUFFER_DISK: - return new DiskBlockFactory(); - case Constants.BLOCK_OUTPUT_BYTEBUFFER: - return new ByteBufferBlockFactory(); - default: - throw new IllegalArgumentException("Unsupported block buffer" + - " \"" + name + "\""); - } - } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java index 6495f1db096ef..822b7c7c25dc1 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java @@ -31,7 +31,6 @@ import java.util.Objects; import java.util.concurrent.ExecutorService; import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicBoolean; import com.amazonaws.AmazonClientException; import com.amazonaws.AmazonServiceException; @@ -118,6 +117,8 @@ public class S3AFileSystem extends FileSystem { private ExecutorService 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; @@ -130,7 +131,7 @@ public class S3AFileSystem extends FileSystem { private static final int MAX_ENTRIES_TO_DELETE = 1000; private boolean blockUploadEnabled; private String blockOutputBuffer; - private S3ADataBlocks.AbstractBlockFactory blockFactory; + private S3ADataBlocks.BlockFactory blockFactory; /* * Register Deprecated options. @@ -220,6 +221,7 @@ public StorageStatistics provide() { serverSideEncryptionAlgorithm = conf.getTrimmed(SERVER_SIDE_ENCRYPTION_ALGORITHM); + LOG.debug("Using encryption {}", serverSideEncryptionAlgorithm); inputPolicy = S3AInputPolicy.getPolicy( conf.getTrimmed(INPUT_FADVISE, INPUT_FADV_NORMAL)); @@ -229,10 +231,11 @@ public StorageStatistics provide() { if (blockUploadEnabled) { partSize = ensureOutputParameterInRange(MULTIPART_SIZE, partSize); - blockFactory = S3ADataBlocks.createFactory(blockOutputBuffer); - blockFactory.init(this); - LOG.debug("Uploading data via Block Upload, buffer = {}", - blockOutputBuffer); + blockFactory = S3ADataBlocks.createFactory(this, blockOutputBuffer); + LOG.debug("Using S3ABlockOutputStream with buffer = {}; block={}", + blockOutputBuffer, partSize); + } else { + LOG.debug("Using S3AOutputStream"); } } catch (AmazonClientException e) { throw translateException("initializing ", new Path(name), e); @@ -1057,7 +1060,7 @@ public void incrementPutCompletedStatistics(boolean success, long bytes) { * @param bytes bytes successfully uploaded. */ public void incrementPutProgressStatistics(String key, long bytes) { - LOG.debug("PUT {}: {} bytes", key, bytes); + PROGRESS.debug("PUT {}: {} bytes", key, bytes); incrementWriteOperations(); if (bytes > 0) { statistics.incrementBytesWritten(bytes); diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java index d86694c9c87b1..100a9e6539a29 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java @@ -118,6 +118,7 @@ public class S3AInstrumentation { private static final Statistic[] GAUGES_TO_CREATE = { OBJECT_PUT_REQUESTS_ACTIVE, OBJECT_PUT_BYTES_PENDING, + STREAM_WRITE_BLOCK_UPLOADS_ACTIVE, STREAM_WRITE_BLOCK_UPLOADS_PENDING, STREAM_WRITE_BLOCK_UPLOADS_DATA_PENDING, STREAM_WRITE_BANDWIDTH, @@ -299,7 +300,7 @@ private MutableCounterLong lookupCounter(String name) { public MutableGaugeLong lookupGauge(String name) { MutableMetric metric = lookupMetric(name); if (metric == null) { - LOG.debug("No metric {}", name); + LOG.debug("No gauge {}", name); } return (MutableGaugeLong) metric; } diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md index 4b117ab663107..8cd4a1c8cf72c 100644 --- a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md +++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md @@ -990,16 +990,17 @@ This is high-performance incremental file upload mechanism which: No effect if fs.s3a.block.upload is false. + - - fs.s3a.multipart.size - 104857600 - - How big (in bytes) to split upload or copy operations up into. - - - + + fs.s3a.multipart.size + 104857600 + + How big (in bytes) to split upload or copy operations up into. + + + ``` **Notes** @@ -1047,7 +1048,7 @@ increases. ```xml - fs.s3a.block.upload + fs.s3a.block.output true @@ -1087,13 +1088,13 @@ kept low. ```xml - fs.s3a.block.upload + fs.s3a.block.output true fs.s3a.block.output.buffer - array + bytebuffer @@ -1126,7 +1127,7 @@ kept low. As an example: three threads and one queued task will consume ```xml - fs.s3a.block.upload + fs.s3a.block.output true diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestDataBlocks.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestDataBlocks.java index 42801b0d87b33..90266aa8211a2 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestDataBlocks.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestDataBlocks.java @@ -48,7 +48,7 @@ public void nameThread() { @Test public void testByteBufferIO() throws Throwable { try(S3ADataBlocks.ByteBufferBlockFactory factory = - new S3ADataBlocks.ByteBufferBlockFactory()) { + new S3ADataBlocks.ByteBufferBlockFactory(null)) { int limit = 128; S3ADataBlocks.ByteBufferBlockFactory.ByteBufferBlock block = factory.create(limit); @@ -67,7 +67,7 @@ public void testByteBufferIO() throws Throwable { // now start the write S3ADataBlocks.ByteBufferBlockFactory.ByteBufferInputStream - stream = block.openForUpload(); + stream = block.startUpload(); assertTrue("!hasRemaining() in " + stream, stream.hasRemaining()); int expected = bufferLen; assertEquals("wrong available() in " + stream,