HADOOP-15607. AliyunOSS: fix duplicated partNumber issue in AliyunOSSBlockOutputStream. Contributed by Jinhu Wu.
This commit is contained in:
parent
21e416ad27
commit
418e957c64
|
@ -33,7 +33,9 @@ import java.io.FileOutputStream;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.io.OutputStream;
|
import java.io.OutputStream;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
|
import java.util.HashMap;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
import java.util.Map;
|
||||||
import java.util.concurrent.Callable;
|
import java.util.concurrent.Callable;
|
||||||
import java.util.concurrent.ExecutionException;
|
import java.util.concurrent.ExecutionException;
|
||||||
import java.util.concurrent.ExecutorService;
|
import java.util.concurrent.ExecutorService;
|
||||||
|
@ -51,7 +53,7 @@ public class AliyunOSSBlockOutputStream extends OutputStream {
|
||||||
private boolean closed;
|
private boolean closed;
|
||||||
private String key;
|
private String key;
|
||||||
private File blockFile;
|
private File blockFile;
|
||||||
private List<File> blockFiles = new ArrayList<>();
|
private Map<Integer, File> blockFiles = new HashMap<>();
|
||||||
private long blockSize;
|
private long blockSize;
|
||||||
private int blockId = 0;
|
private int blockId = 0;
|
||||||
private long blockWritten = 0L;
|
private long blockWritten = 0L;
|
||||||
|
@ -95,8 +97,9 @@ public class AliyunOSSBlockOutputStream extends OutputStream {
|
||||||
|
|
||||||
blockStream.flush();
|
blockStream.flush();
|
||||||
blockStream.close();
|
blockStream.close();
|
||||||
if (!blockFiles.contains(blockFile)) {
|
if (!blockFiles.values().contains(blockFile)) {
|
||||||
blockFiles.add(blockFile);
|
blockId++;
|
||||||
|
blockFiles.put(blockId, blockFile);
|
||||||
}
|
}
|
||||||
|
|
||||||
try {
|
try {
|
||||||
|
@ -110,7 +113,7 @@ public class AliyunOSSBlockOutputStream extends OutputStream {
|
||||||
@Override
|
@Override
|
||||||
public PartETag call() throws Exception {
|
public PartETag call() throws Exception {
|
||||||
PartETag partETag = store.uploadPart(blockFile, key, uploadId,
|
PartETag partETag = store.uploadPart(blockFile, key, uploadId,
|
||||||
blockId + 1);
|
blockId);
|
||||||
return partETag;
|
return partETag;
|
||||||
}
|
}
|
||||||
});
|
});
|
||||||
|
@ -124,11 +127,7 @@ public class AliyunOSSBlockOutputStream extends OutputStream {
|
||||||
store.completeMultipartUpload(key, uploadId, partETags);
|
store.completeMultipartUpload(key, uploadId, partETags);
|
||||||
}
|
}
|
||||||
} finally {
|
} finally {
|
||||||
for (File tFile: blockFiles) {
|
removePartFiles();
|
||||||
if (tFile.exists() && !tFile.delete()) {
|
|
||||||
LOG.warn("Failed to delete temporary file {}", tFile);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
closed = true;
|
closed = true;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -145,41 +144,55 @@ public class AliyunOSSBlockOutputStream extends OutputStream {
|
||||||
if (closed) {
|
if (closed) {
|
||||||
throw new IOException("Stream closed.");
|
throw new IOException("Stream closed.");
|
||||||
}
|
}
|
||||||
try {
|
blockStream.write(b, off, len);
|
||||||
blockStream.write(b, off, len);
|
blockWritten += len;
|
||||||
blockWritten += len;
|
if (blockWritten >= blockSize) {
|
||||||
if (blockWritten >= blockSize) {
|
uploadCurrentPart();
|
||||||
uploadCurrentPart();
|
blockWritten = 0L;
|
||||||
blockWritten = 0L;
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private void removePartFiles() throws IOException {
|
||||||
|
for (ListenableFuture<PartETag> partETagFuture : partETagsFutures) {
|
||||||
|
if (!partETagFuture.isDone()) {
|
||||||
|
continue;
|
||||||
}
|
}
|
||||||
} finally {
|
|
||||||
for (File tFile: blockFiles) {
|
try {
|
||||||
if (tFile.exists() && !tFile.delete()) {
|
File blockFile = blockFiles.get(partETagFuture.get().getPartNumber());
|
||||||
LOG.warn("Failed to delete temporary file {}", tFile);
|
if (blockFile != null && blockFile.exists() && !blockFile.delete()) {
|
||||||
|
LOG.warn("Failed to delete temporary file {}", blockFile);
|
||||||
}
|
}
|
||||||
|
} catch (InterruptedException | ExecutionException e) {
|
||||||
|
throw new IOException(e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private void uploadCurrentPart() throws IOException {
|
private void uploadCurrentPart() throws IOException {
|
||||||
blockFiles.add(blockFile);
|
|
||||||
blockStream.flush();
|
blockStream.flush();
|
||||||
blockStream.close();
|
blockStream.close();
|
||||||
if (blockId == 0) {
|
if (blockId == 0) {
|
||||||
uploadId = store.getUploadId(key);
|
uploadId = store.getUploadId(key);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
blockId++;
|
||||||
|
blockFiles.put(blockId, blockFile);
|
||||||
|
|
||||||
|
final File currentFile = blockFile;
|
||||||
|
final int currentBlockId = blockId;
|
||||||
ListenableFuture<PartETag> partETagFuture =
|
ListenableFuture<PartETag> partETagFuture =
|
||||||
executorService.submit(new Callable<PartETag>() {
|
executorService.submit(new Callable<PartETag>() {
|
||||||
@Override
|
@Override
|
||||||
public PartETag call() throws Exception {
|
public PartETag call() throws Exception {
|
||||||
PartETag partETag = store.uploadPart(blockFile, key, uploadId,
|
PartETag partETag = store.uploadPart(currentFile, key, uploadId,
|
||||||
blockId + 1);
|
currentBlockId);
|
||||||
return partETag;
|
return partETag;
|
||||||
}
|
}
|
||||||
});
|
});
|
||||||
partETagsFutures.add(partETagFuture);
|
partETagsFutures.add(partETagFuture);
|
||||||
|
removePartFiles();
|
||||||
blockFile = newBlockFile();
|
blockFile = newBlockFile();
|
||||||
blockId++;
|
|
||||||
blockStream = new BufferedOutputStream(new FileOutputStream(blockFile));
|
blockStream = new BufferedOutputStream(new FileOutputStream(blockFile));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -450,6 +450,8 @@ public class AliyunOSSFileSystemStore {
|
||||||
request.setRange(byteStart, byteEnd);
|
request.setRange(byteStart, byteEnd);
|
||||||
return ossClient.getObject(request).getObjectContent();
|
return ossClient.getObject(request).getObjectContent();
|
||||||
} catch (OSSException | ClientException e) {
|
} catch (OSSException | ClientException e) {
|
||||||
|
LOG.error("Exception thrown when store retrieves key: "
|
||||||
|
+ key + ", exception: " + e);
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -31,6 +31,7 @@ import org.junit.rules.Timeout;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
|
||||||
import static org.apache.hadoop.fs.aliyun.oss.Constants.MULTIPART_UPLOAD_PART_SIZE_DEFAULT;
|
import static org.apache.hadoop.fs.aliyun.oss.Constants.MULTIPART_UPLOAD_PART_SIZE_DEFAULT;
|
||||||
|
import static org.apache.hadoop.fs.contract.ContractTestUtils.IO_CHUNK_BUFFER_SIZE;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Tests regular and multi-part upload functionality for
|
* Tests regular and multi-part upload functionality for
|
||||||
|
@ -48,7 +49,10 @@ public class TestAliyunOSSBlockOutputStream {
|
||||||
public void setUp() throws Exception {
|
public void setUp() throws Exception {
|
||||||
Configuration conf = new Configuration();
|
Configuration conf = new Configuration();
|
||||||
conf.setLong(Constants.MIN_MULTIPART_UPLOAD_THRESHOLD_KEY, 5 * 1024 * 1024);
|
conf.setLong(Constants.MIN_MULTIPART_UPLOAD_THRESHOLD_KEY, 5 * 1024 * 1024);
|
||||||
conf.setInt(Constants.MULTIPART_UPLOAD_PART_SIZE_KEY, 5 * 1024 * 1024);
|
conf.setInt(Constants.MULTIPART_UPLOAD_PART_SIZE_KEY, 1024 * 1024);
|
||||||
|
conf.setInt(IO_CHUNK_BUFFER_SIZE,
|
||||||
|
conf.getInt(Constants.MULTIPART_UPLOAD_PART_SIZE_KEY, 0));
|
||||||
|
conf.setInt(Constants.UPLOAD_ACTIVE_BLOCKS_KEY, 20);
|
||||||
fs = AliyunOSSTestUtils.createTestFileSystem(conf);
|
fs = AliyunOSSTestUtils.createTestFileSystem(conf);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -84,6 +88,12 @@ public class TestAliyunOSSBlockOutputStream {
|
||||||
6 * 1024 * 1024 + 1);
|
6 * 1024 * 1024 + 1);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testMultiPartUploadConcurrent() throws IOException {
|
||||||
|
ContractTestUtils.createAndVerifyFile(fs, getTestPath(),
|
||||||
|
50 * 1024 * 1024 - 1);
|
||||||
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testHugeUpload() throws IOException {
|
public void testHugeUpload() throws IOException {
|
||||||
ContractTestUtils.createAndVerifyFile(fs, getTestPath(),
|
ContractTestUtils.createAndVerifyFile(fs, getTestPath(),
|
||||||
|
|
Loading…
Reference in New Issue