|
@@ -180,7 +180,7 @@ final class S3ADataBlocks {
|
|
|
* @param statistics stats to work with
|
|
|
* @return a new block.
|
|
|
*/
|
|
|
- abstract DataBlock create(long index, int limit,
|
|
|
+ abstract DataBlock create(long index, long limit,
|
|
|
BlockOutputStreamStatistics statistics)
|
|
|
throws IOException;
|
|
|
|
|
@@ -258,7 +258,7 @@ final class S3ADataBlocks {
|
|
|
* Return the current data size.
|
|
|
* @return the size of the data
|
|
|
*/
|
|
|
- abstract int dataSize();
|
|
|
+ abstract long dataSize();
|
|
|
|
|
|
/**
|
|
|
* Predicate to verify that the block has the capacity to write
|
|
@@ -280,7 +280,7 @@ final class S3ADataBlocks {
|
|
|
* The remaining capacity in the block before it is full.
|
|
|
* @return the number of bytes remaining.
|
|
|
*/
|
|
|
- abstract int remainingCapacity();
|
|
|
+ abstract long remainingCapacity();
|
|
|
|
|
|
/**
|
|
|
* Write a series of bytes from the buffer, from the offset.
|
|
@@ -391,9 +391,11 @@ final class S3ADataBlocks {
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
- DataBlock create(long index, int limit,
|
|
|
+ DataBlock create(long index, long limit,
|
|
|
BlockOutputStreamStatistics statistics)
|
|
|
throws IOException {
|
|
|
+ Preconditions.checkArgument(limit > 0,
|
|
|
+ "Invalid block size: %d", limit);
|
|
|
return new ByteArrayBlock(0, limit, statistics);
|
|
|
}
|
|
|
|
|
@@ -436,11 +438,11 @@ final class S3ADataBlocks {
|
|
|
private Integer dataSize;
|
|
|
|
|
|
ByteArrayBlock(long index,
|
|
|
- int limit,
|
|
|
+ long limit,
|
|
|
BlockOutputStreamStatistics statistics) {
|
|
|
super(index, statistics);
|
|
|
- this.limit = limit;
|
|
|
- buffer = new S3AByteArrayOutputStream(limit);
|
|
|
+ this.limit = (limit > Integer.MAX_VALUE) ? Integer.MAX_VALUE : (int) limit;
|
|
|
+ buffer = new S3AByteArrayOutputStream(this.limit);
|
|
|
blockAllocated();
|
|
|
}
|
|
|
|
|
@@ -449,7 +451,7 @@ final class S3ADataBlocks {
|
|
|
* @return the amount of data available to upload.
|
|
|
*/
|
|
|
@Override
|
|
|
- int dataSize() {
|
|
|
+ long dataSize() {
|
|
|
return dataSize != null ? dataSize : buffer.size();
|
|
|
}
|
|
|
|
|
@@ -468,14 +470,14 @@ final class S3ADataBlocks {
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
- int remainingCapacity() {
|
|
|
+ long remainingCapacity() {
|
|
|
return limit - dataSize();
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
int write(byte[] b, int offset, int len) throws IOException {
|
|
|
super.write(b, offset, len);
|
|
|
- int written = Math.min(remainingCapacity(), len);
|
|
|
+ int written = (int) Math.min(remainingCapacity(), len);
|
|
|
buffer.write(b, offset, written);
|
|
|
return written;
|
|
|
}
|
|
@@ -514,9 +516,11 @@ final class S3ADataBlocks {
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
- ByteBufferBlock create(long index, int limit,
|
|
|
+ ByteBufferBlock create(long index, long limit,
|
|
|
BlockOutputStreamStatistics statistics)
|
|
|
throws IOException {
|
|
|
+ Preconditions.checkArgument(limit > 0,
|
|
|
+ "Invalid block size: %d", limit);
|
|
|
return new ByteBufferBlock(index, limit, statistics);
|
|
|
}
|
|
|
|
|
@@ -564,11 +568,12 @@ final class S3ADataBlocks {
|
|
|
* @param statistics statistics to update
|
|
|
*/
|
|
|
ByteBufferBlock(long index,
|
|
|
- int bufferSize,
|
|
|
+ long bufferSize,
|
|
|
BlockOutputStreamStatistics statistics) {
|
|
|
super(index, statistics);
|
|
|
- this.bufferSize = bufferSize;
|
|
|
- blockBuffer = requestBuffer(bufferSize);
|
|
|
+ this.bufferSize = bufferSize > Integer.MAX_VALUE ?
|
|
|
+ Integer.MAX_VALUE : (int) bufferSize;
|
|
|
+ blockBuffer = requestBuffer(this.bufferSize);
|
|
|
blockAllocated();
|
|
|
}
|
|
|
|
|
@@ -577,7 +582,7 @@ final class S3ADataBlocks {
|
|
|
* @return the amount of data available to upload.
|
|
|
*/
|
|
|
@Override
|
|
|
- int dataSize() {
|
|
|
+ long dataSize() {
|
|
|
return dataSize != null ? dataSize : bufferCapacityUsed();
|
|
|
}
|
|
|
|
|
@@ -598,7 +603,7 @@ final class S3ADataBlocks {
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
- public int remainingCapacity() {
|
|
|
+ public long remainingCapacity() {
|
|
|
return blockBuffer != null ? blockBuffer.remaining() : 0;
|
|
|
}
|
|
|
|
|
@@ -609,7 +614,7 @@ final class S3ADataBlocks {
|
|
|
@Override
|
|
|
int write(byte[] b, int offset, int len) throws IOException {
|
|
|
super.write(b, offset, len);
|
|
|
- int written = Math.min(remainingCapacity(), len);
|
|
|
+ int written = (int) Math.min(remainingCapacity(), len);
|
|
|
blockBuffer.put(b, offset, written);
|
|
|
return written;
|
|
|
}
|
|
@@ -802,16 +807,18 @@ final class S3ADataBlocks {
|
|
|
* Create a temp file and a {@link DiskBlock} instance to manage it.
|
|
|
*
|
|
|
* @param index block index
|
|
|
- * @param limit limit of the block.
|
|
|
+ * @param limit limit of the block. -1 means "no limit"
|
|
|
* @param statistics statistics to update
|
|
|
* @return the new block
|
|
|
* @throws IOException IO problems
|
|
|
*/
|
|
|
@Override
|
|
|
DataBlock create(long index,
|
|
|
- int limit,
|
|
|
+ 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());
|
|
@@ -825,14 +832,14 @@ final class S3ADataBlocks {
|
|
|
*/
|
|
|
static class DiskBlock extends DataBlock {
|
|
|
|
|
|
- private int bytesWritten;
|
|
|
+ private long bytesWritten;
|
|
|
private final File bufferFile;
|
|
|
- private final int limit;
|
|
|
+ private final long limit;
|
|
|
private BufferedOutputStream out;
|
|
|
private final AtomicBoolean closed = new AtomicBoolean(false);
|
|
|
|
|
|
DiskBlock(File bufferFile,
|
|
|
- int limit,
|
|
|
+ long limit,
|
|
|
long index,
|
|
|
BlockOutputStreamStatistics statistics)
|
|
|
throws FileNotFoundException {
|
|
@@ -844,24 +851,39 @@ final class S3ADataBlocks {
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
- int dataSize() {
|
|
|
+ long dataSize() {
|
|
|
return bytesWritten;
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * Does this block have unlimited space?
|
|
|
+ * @return true if a block with no size limit was created.
|
|
|
+ */
|
|
|
+ private boolean unlimited() {
|
|
|
+ return limit < 0;
|
|
|
+ }
|
|
|
+
|
|
|
@Override
|
|
|
boolean hasCapacity(long bytes) {
|
|
|
- return dataSize() + bytes <= limit;
|
|
|
+ return unlimited() || dataSize() + bytes <= limit;
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * {@inheritDoc}.
|
|
|
+ * If there is no limit to capacity, return MAX_VALUE.
|
|
|
+ * @return capacity in the block.
|
|
|
+ */
|
|
|
@Override
|
|
|
- int remainingCapacity() {
|
|
|
- return limit - bytesWritten;
|
|
|
+ long remainingCapacity() {
|
|
|
+ return unlimited()
|
|
|
+ ? Integer.MAX_VALUE
|
|
|
+ : limit - bytesWritten;
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
int write(byte[] b, int offset, int len) throws IOException {
|
|
|
super.write(b, offset, len);
|
|
|
- int written = Math.min(remainingCapacity(), len);
|
|
|
+ int written = (int) Math.min(remainingCapacity(), len);
|
|
|
out.write(b, offset, written);
|
|
|
bytesWritten += written;
|
|
|
return written;
|