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 43a2b7e0dbd5b..2febc87aec36d 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 @@ -232,7 +232,9 @@ private synchronized S3ADataBlocks.DataBlock createBlockIfNeeded() LOG.error("Number of partitions in stream exceeds limit for S3: " + Constants.MAX_MULTIPART_COUNT + " write may fail."); } - activeBlock = blockFactory.create(blockCount, this.blockSize, statistics); + activeBlock = blockFactory.create( + writeOperationHelper.getAuditSpan().getSpanId(), + key, blockCount, this.blockSize, statistics); } return activeBlock; } @@ -728,8 +730,14 @@ public void hsync() throws IOException { /** * Shared processing of Syncable operation reporting/downgrade. + * + * Syncable API is not supported, so calls to hsync/hflush will throw an + * UnsupportedOperationException unless the stream was constructed with + * {@link #downgradeSyncableExceptions} set to true, in which case the stream is flushed. + * @throws IOException IO Problem + * @throws UnsupportedOperationException if downgrade syncable exceptions is set to false */ - private void handleSyncableInvocation() { + private void handleSyncableInvocation() throws IOException { final UnsupportedOperationException ex = new UnsupportedOperationException(E_NOT_SYNCABLE); if (!downgradeSyncableExceptions) { @@ -741,6 +749,7 @@ private void handleSyncableInvocation() { key); // and log at debug LOG.debug("Downgrading Syncable call", ex); + flush(); } @Override 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 b20d8e859aa88..2299892b35b94 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 @@ -175,12 +175,14 @@ protected BlockFactory(S3AFileSystem owner) { /** * Create a block. * + * @param spanId id of the audit span + * @param key key of s3 object being written to * @param index index of block * @param limit limit of the block. * @param statistics stats to work with * @return a new block. */ - abstract DataBlock create(long index, long limit, + abstract DataBlock create(String spanId, String key, long index, long limit, BlockOutputStreamStatistics statistics) throws IOException; @@ -391,11 +393,11 @@ static class ArrayBlockFactory extends BlockFactory { } @Override - DataBlock create(long index, long limit, + DataBlock create(String spanId, String key, long index, long limit, BlockOutputStreamStatistics statistics) throws IOException { Preconditions.checkArgument(limit > 0, - "Invalid block size: %d", limit); + "Invalid block size: %d [%s]", limit, key); return new ByteArrayBlock(0, limit, statistics); } @@ -516,11 +518,11 @@ static class ByteBufferBlockFactory extends BlockFactory { } @Override - ByteBufferBlock create(long index, long limit, + ByteBufferBlock create(String spanId, String key, long index, long limit, BlockOutputStreamStatistics statistics) throws IOException { Preconditions.checkArgument(limit > 0, - "Invalid block size: %d", limit); + "Invalid block size: %d [%s]", limit, key); return new ByteBufferBlock(index, limit, statistics); } @@ -798,6 +800,8 @@ public String toString() { * Buffer blocks to disk. */ static class DiskBlockFactory extends BlockFactory { + private static final String ESCAPED_FORWARD_SLASH = "EFS"; + private static final String ESCAPED_BACKSLASH = "EBS"; DiskBlockFactory(S3AFileSystem owner) { super(owner); @@ -806,6 +810,8 @@ static class DiskBlockFactory extends BlockFactory { /** * Create a temp file and a {@link DiskBlock} instance to manage it. * + * @param spanId id of the audit span + * @param key of the s3 object being written * @param index block index * @param limit limit of the block. -1 means "no limit" * @param statistics statistics to update @@ -813,17 +819,22 @@ static class DiskBlockFactory extends BlockFactory { * @throws IOException IO problems */ @Override - DataBlock create(long index, + DataBlock create(String spanId, String key, long index, long limit, BlockOutputStreamStatistics statistics) throws IOException { Preconditions.checkArgument(limit != 0, - "Invalid block size: %d", limit); - File destFile = getOwner() - .createTmpFileForWrite(String.format("s3ablock-%04d-", index), - limit, getOwner().getConf()); + "Invalid block size: %d [%s]", limit, key); + String prefix = String.format("s3ablock-%04d-%s-%s-", index, spanId, escapeS3Key(key)); + File destFile = getOwner().createTmpFileForWrite(prefix, limit, getOwner().getConf()); return new DiskBlock(destFile, limit, index, statistics); } + + protected static String escapeS3Key(String key) { + return key + .replace("\\", ESCAPED_BACKSLASH) + .replace("/", ESCAPED_FORWARD_SLASH); + } } /** 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 999186f8cd5ae..f985261f64c6e 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 @@ -1369,12 +1369,69 @@ public S3AEncryptionMethods getS3EncryptionAlgorithm() { File createTmpFileForWrite(String pathStr, long size, Configuration conf) throws IOException { initLocalDirAllocatorIfNotInitialized(conf); - Path path = directoryAllocator.getLocalPathForWrite(pathStr, - size, conf); + Path path = directoryAllocator.getLocalPathForWrite(pathStr, size, conf); File dir = new File(path.getParent().toUri().getPath()); - String prefix = path.getName(); - // create a temp file on this directory - return File.createTempFile(prefix, null, dir); + return safeCreateTempFile(pathStr, null, dir); + } + + // TODO remove this method when hadoop upgrades to a newer version of java than 1.8 + /** + * Ensure that the temp file prefix and suffix don't exceed the maximum number of characters + * allowed by the underlying file system. This validation isn't required in Java 9+ since + * {@link java.io.File#createTempFile(String, String, File)} automatically truncates file names. + * + * @param prefix prefix for the temporary file + * @param suffix suffix for the temporary file + * @param dir directory to create the temporary file in + * @return a unique temporary file + * @throws IOException + */ + static File safeCreateTempFile(String prefix, String suffix, File dir) throws IOException + { + // avoid validating multiple times. + // if the jvm running is version 9+ then defer to java.io.File validation implementation + if(Float.parseFloat(System.getProperty("java.class.version")) >= 53) { + return File.createTempFile(prefix, null, dir); + } + + // if no suffix was defined assume the default + if(suffix == null) { + suffix = ".tmp"; + } + // Use only the file name from the supplied prefix + prefix = (new File(prefix)).getName(); + + int prefixLength = prefix.length(); + int suffixLength = suffix.length(); + int maxRandomSuffixLen = 19; // Long.toUnsignedString(Long.MAX_VALUE).length() + + String name; + int nameMax = 255; // unable to access the underlying FS directly, so assume 255 + int excess = prefixLength + maxRandomSuffixLen + suffixLength - nameMax; + + // shorten the prefix length if the file name exceeds 255 chars + if (excess > 0) { + // Attempt to shorten the prefix length to no less than 3 + prefixLength = shortenSubName(prefixLength, excess, 3); + prefix = prefix.substring(0, prefixLength); + } + // shorten the suffix if the file name still exceeds 255 chars + excess = prefixLength + maxRandomSuffixLen + suffixLength - nameMax; + if (excess > 0) { + // Attempt to shorten the suffix length to no less than 3 + suffixLength = shortenSubName(suffixLength, excess, 3); + suffix = suffix.substring(0, suffixLength); + } + + return File.createTempFile(prefix, suffix, dir); + } + + private static int shortenSubName(int subNameLength, int excess, int nameMin) { + int newLength = Math.max(nameMin, subNameLength - excess); + if (newLength < subNameLength) { + return newLength; + } + return subNameLength; } /** diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperationHelper.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperationHelper.java index 8e15a10944c38..3f42d2caf4a0b 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperationHelper.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperationHelper.java @@ -217,6 +217,7 @@ public T retry(String action, * Get the audit span this object was created with. * @return the audit span */ + @Override public AuditSpan getAuditSpan() { return auditSpan; } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperations.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperations.java index bd6e5ac14731f..0c34371983595 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperations.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperations.java @@ -43,6 +43,7 @@ import org.apache.hadoop.fs.PathIOException; import org.apache.hadoop.fs.s3a.impl.PutObjectOptions; import org.apache.hadoop.fs.statistics.DurationTrackerFactory; +import org.apache.hadoop.fs.store.audit.AuditSpan; import org.apache.hadoop.fs.store.audit.AuditSpanSource; import org.apache.hadoop.util.functional.CallableRaisingIOE; @@ -305,6 +306,12 @@ UploadPartResult uploadPart(UploadPartRequest request, */ Configuration getConf(); + /** + * Get the audit span this object was created with. + * @return the audit span + */ + AuditSpan getAuditSpan(); + /** * Create a S3 Select request for the destination path. * This does not build the query. diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ABlockOutputArray.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ABlockOutputArray.java index 53fa0d83b55a7..bfeaa25f16ca5 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ABlockOutputArray.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ABlockOutputArray.java @@ -29,9 +29,12 @@ import org.junit.BeforeClass; import org.junit.Test; +import java.io.File; import java.io.IOException; import java.io.InputStream; import java.net.URI; +import java.util.Arrays; +import java.util.Objects; import static org.apache.hadoop.fs.StreamCapabilities.ABORTABLE_STREAM; import static org.apache.hadoop.fs.s3a.Constants.*; @@ -79,6 +82,46 @@ public void testRegularUpload() throws IOException { verifyUpload("regular", 1024); } + /** + * Test that the DiskBlock's local file doesn't result in error when the S3 key exceeds the max + * char limit of the local file system. Currently + * {@link java.io.File#createTempFile(String, String, File)} is being relied on to handle the + * truncation. + * @throws IOException + */ + @Test + public void testDiskBlockCreate() throws IOException { + String s3Key = // 1024 char + "very_long_s3_key__very_long_s3_key__very_long_s3_key__very_long_s3_key__" + + "very_long_s3_key__very_long_s3_key__very_long_s3_key__very_long_s3_key__" + + "very_long_s3_key__very_long_s3_key__very_long_s3_key__very_long_s3_key__" + + "very_long_s3_key__very_long_s3_key__very_long_s3_key__very_long_s3_key__" + + "very_long_s3_key__very_long_s3_key__very_long_s3_key__very_long_s3_key__" + + "very_long_s3_key__very_long_s3_key__very_long_s3_key__very_long_s3_key__" + + "very_long_s3_key__very_long_s3_key__very_long_s3_key__very_long_s3_key__" + + "very_long_s3_key__very_long_s3_key__very_long_s3_key__very_long_s3_key__" + + "very_long_s3_key__very_long_s3_key__very_long_s3_key__very_long_s3_key__" + + "very_long_s3_key__very_long_s3_key__very_long_s3_key__very_long_s3_key__" + + "very_long_s3_key__very_long_s3_key__very_long_s3_key__very_long_s3_key__" + + "very_long_s3_key__very_long_s3_key__very_long_s3_key__very_long_s3_key__" + + "very_long_s3_key__very_long_s3_key__very_long_s3_key__very_long_s3_key__" + + "very_long_s3_key__very_long_s3_key__very_long_s3_key__very_long_s3_key__" + + "very_long_s3_key"; + long blockSize = getFileSystem().getDefaultBlockSize(); + try (S3ADataBlocks.BlockFactory diskBlockFactory = + new S3ADataBlocks.DiskBlockFactory(getFileSystem()); + S3ADataBlocks.DataBlock dataBlock = + diskBlockFactory.create("spanId", s3Key, 1, blockSize, null); + ) { + String tmpDir = getConfiguration().get("hadoop.tmp.dir"); + boolean created = Arrays.stream( + Objects.requireNonNull(new File(tmpDir).listFiles())) + .anyMatch(f -> f.getName().contains("very_long_s3_key")); + assertTrue(String.format("tmp file should have been created locally in %s", tmpDir), created); + LOG.info(dataBlock.toString()); // block file name/location can be viewed in failsafe-report + } + } + @Test(expected = IOException.class) public void testWriteAfterStreamClose() throws Throwable { Path dest = path("testWriteAfterStreamClose"); @@ -136,7 +179,7 @@ private void markAndResetDatablock(S3ADataBlocks.BlockFactory factory) new S3AInstrumentation(new URI("s3a://example")); BlockOutputStreamStatistics outstats = instrumentation.newOutputStreamStatistics(null); - S3ADataBlocks.DataBlock block = factory.create(1, BLOCK_SIZE, outstats); + S3ADataBlocks.DataBlock block = factory.create("spanId", "object/key", 1, BLOCK_SIZE, outstats); block.write(dataset, 0, dataset.length); S3ADataBlocks.BlockUploadData uploadData = block.startUpload(); InputStream stream = uploadData.getUploadStream(); 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 700ef5ced3d8a..d2ea0218acd9f 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 @@ -51,7 +51,7 @@ public void testByteBufferIO() throws Throwable { new S3ADataBlocks.ByteBufferBlockFactory(null)) { int limit = 128; S3ADataBlocks.ByteBufferBlockFactory.ByteBufferBlock block - = factory.create(1, limit, null); + = factory.create("spanId", "s3\\object/key", 1, limit, null); assertOutstandingBuffers(factory, 1); byte[] buffer = ContractTestUtils.toAsciiByteArray("test data"); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3ABlockOutputStream.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3ABlockOutputStream.java index ffa2c81e58adf..beeeb39fc4852 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3ABlockOutputStream.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3ABlockOutputStream.java @@ -26,6 +26,7 @@ import org.apache.hadoop.fs.s3a.statistics.impl.EmptyS3AStatisticsContext; import org.apache.hadoop.fs.s3a.test.MinimalWriteOperationHelperCallbacks; import org.apache.hadoop.fs.statistics.IOStatisticsContext; +import org.apache.hadoop.fs.store.audit.AuditSpan; import org.apache.hadoop.util.Progressable; import org.junit.Before; import org.junit.Test; @@ -38,7 +39,10 @@ import static org.apache.hadoop.test.LambdaTestUtils.intercept; import static org.mockito.Mockito.doThrow; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.never; import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; /** @@ -59,6 +63,9 @@ private S3ABlockOutputStream.BlockOutputStreamBuilder mockS3ABuilder() { mock(S3ADataBlocks.BlockFactory.class); long blockSize = Constants.DEFAULT_MULTIPART_SIZE; WriteOperationHelper oHelper = mock(WriteOperationHelper.class); + AuditSpan auditSpan = mock(AuditSpan.class); + when(auditSpan.getSpanId()).thenReturn("spanId"); + when(oHelper.getAuditSpan()).thenReturn(auditSpan); PutTracker putTracker = mock(PutTracker.class); final S3ABlockOutputStream.BlockOutputStreamBuilder builder = S3ABlockOutputStream.builder() @@ -156,6 +163,7 @@ public void testSyncableUnsupported() throws Exception { stream = spy(new S3ABlockOutputStream(builder)); intercept(UnsupportedOperationException.class, () -> stream.hflush()); intercept(UnsupportedOperationException.class, () -> stream.hsync()); + verify(stream, never()).flush(); } /** @@ -169,8 +177,11 @@ public void testSyncableDowngrade() throws Exception { builder.withDowngradeSyncableExceptions(true); stream = spy(new S3ABlockOutputStream(builder)); + verify(stream, never()).flush(); stream.hflush(); + verify(stream, times(1)).flush(); stream.hsync(); + verify(stream, times(2)).flush(); } } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AFileSystem.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AFileSystem.java new file mode 100644 index 0000000000000..5d3331be88b67 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AFileSystem.java @@ -0,0 +1,110 @@ +/* + * 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.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.Timeout; + +import java.io.File; +import java.io.IOException; +import java.nio.file.Files; +import java.util.regex.Pattern; + +/** + * Unit tests for {@link S3AFileSystem}. + */ +public class TestS3AFileSystem extends Assert { + final File TEMP_DIR = new File("target/build/test/TestS3AFileSystem"); + final String longStr = // 1024 char + "very_long_s3_key__very_long_s3_key__very_long_s3_key__very_long_s3_key__" + + "very_long_s3_key__very_long_s3_key__very_long_s3_key__very_long_s3_key__" + + "very_long_s3_key__very_long_s3_key__very_long_s3_key__very_long_s3_key__" + + "very_long_s3_key__very_long_s3_key__very_long_s3_key__very_long_s3_key__" + + "very_long_s3_key__very_long_s3_key__very_long_s3_key__very_long_s3_key__" + + "very_long_s3_key__very_long_s3_key__very_long_s3_key__very_long_s3_key__" + + "very_long_s3_key__very_long_s3_key__very_long_s3_key__very_long_s3_key__" + + "very_long_s3_key__very_long_s3_key__very_long_s3_key__very_long_s3_key__" + + "very_long_s3_key__very_long_s3_key__very_long_s3_key__very_long_s3_key__" + + "very_long_s3_key__very_long_s3_key__very_long_s3_key__very_long_s3_key__" + + "very_long_s3_key__very_long_s3_key__very_long_s3_key__very_long_s3_key__" + + "very_long_s3_key__very_long_s3_key__very_long_s3_key__very_long_s3_key__" + + "very_long_s3_key__very_long_s3_key__very_long_s3_key__very_long_s3_key__" + + "very_long_s3_key__very_long_s3_key__very_long_s3_key__very_long_s3_key__" + + "very_long_s3_key"; + final String longStrTruncated = "very_long_s3_key__very_long_s3_key__"; + + @Rule + public Timeout testTimeout = new Timeout(30 * 1000); + + @Before + public void init() throws IOException { + Files.createDirectories(TEMP_DIR.toPath()); + } + + @After + public void teardown() throws IOException { + File[] testOutputFiles = TEMP_DIR.listFiles(); + for(File file: testOutputFiles) { + Files.delete(file.toPath()); + } + Files.deleteIfExists(TEMP_DIR.toPath()); + } + + @Before + public void nameThread() { + Thread.currentThread().setName("JUnit"); + } + + /** + * Test the {@link S3AFileSystem#safeCreateTempFile(String, String, File)}. + * The code verifies that the input prefix and suffix don't exceed the file system's max name + * length and cause an exception. + * + * This test verifies the basic contract of the process. + */ + @Test + public void testSafeCreateTempFile() throws Throwable { + // fitting name isn't changed + File noChangesRequired = S3AFileSystem.safeCreateTempFile("noChangesRequired", ".tmp", TEMP_DIR); + assertTrue(noChangesRequired.exists()); + String noChangesRequiredName = noChangesRequired.getName(); + assertTrue(noChangesRequiredName.startsWith("noChangesRequired")); + assertTrue(noChangesRequiredName.endsWith(".tmp")); + + // a long prefix should be truncated + File excessivelyLongPrefix = S3AFileSystem.safeCreateTempFile(longStr, ".tmp", TEMP_DIR); + assertTrue(excessivelyLongPrefix.exists()); + String excessivelyLongPrefixName = excessivelyLongPrefix.getName(); + assertTrue(excessivelyLongPrefixName.startsWith(longStrTruncated)); + assertTrue(excessivelyLongPrefixName.endsWith(".tmp")); + + // a long suffix should be truncated + File excessivelyLongSuffix = S3AFileSystem.safeCreateTempFile("excessivelyLongSuffix", "." + longStr, TEMP_DIR); + assertTrue(excessivelyLongSuffix.exists()); + String excessivelyLongSuffixName = excessivelyLongSuffix.getName(); + // the prefix should have been truncated first + assertTrue(excessivelyLongSuffixName.startsWith("exc")); + Pattern p = Pattern.compile("^exc\\d{1,19}\\.very_long_s3_key__very_long_s3_key__"); + assertTrue(p.matcher(excessivelyLongSuffixName).find()); + } +}