|
@@ -22,7 +22,11 @@ import java.io.FileOutputStream;
|
|
|
import java.io.IOException;
|
|
|
import java.io.OutputStream;
|
|
|
import java.io.RandomAccessFile;
|
|
|
+import java.util.concurrent.TimeUnit;
|
|
|
import java.util.concurrent.atomic.AtomicReference;
|
|
|
+import java.util.concurrent.locks.Condition;
|
|
|
+import java.util.concurrent.locks.Lock;
|
|
|
+import java.util.concurrent.locks.ReentrantLock;
|
|
|
|
|
|
import org.apache.hadoop.hdfs.protocol.Block;
|
|
|
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
|
|
@@ -42,7 +46,11 @@ import org.apache.hadoop.util.StringUtils;
|
|
|
* The base class implements a temporary replica
|
|
|
*/
|
|
|
public class LocalReplicaInPipeline extends LocalReplica
|
|
|
- implements ReplicaInPipeline {
|
|
|
+ implements ReplicaInPipeline {
|
|
|
+
|
|
|
+ private final Lock lock = new ReentrantLock();
|
|
|
+ private final Condition bytesOnDiskChange = lock.newCondition();
|
|
|
+
|
|
|
private long bytesAcked;
|
|
|
private long bytesOnDisk;
|
|
|
private byte[] lastChecksum;
|
|
@@ -167,16 +175,47 @@ public class LocalReplicaInPipeline extends LocalReplica
|
|
|
bytesReserved = 0;
|
|
|
}
|
|
|
|
|
|
- @Override // ReplicaInPipeline
|
|
|
- public synchronized void setLastChecksumAndDataLen(long dataLength,
|
|
|
- byte[] checksum) {
|
|
|
- this.bytesOnDisk = dataLength;
|
|
|
- this.lastChecksum = checksum;
|
|
|
+ @Override
|
|
|
+ public void setLastChecksumAndDataLen(long dataLength, byte[] checksum) {
|
|
|
+ lock.lock();
|
|
|
+ try {
|
|
|
+ this.bytesOnDisk = dataLength;
|
|
|
+ this.lastChecksum = checksum;
|
|
|
+ bytesOnDiskChange.signalAll();
|
|
|
+ } finally {
|
|
|
+ lock.unlock();
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
- @Override // ReplicaInPipeline
|
|
|
- public synchronized ChunkChecksum getLastChecksumAndDataLen() {
|
|
|
- return new ChunkChecksum(getBytesOnDisk(), lastChecksum);
|
|
|
+ @Override
|
|
|
+ public ChunkChecksum getLastChecksumAndDataLen() {
|
|
|
+ lock.lock();
|
|
|
+ try {
|
|
|
+ return new ChunkChecksum(getBytesOnDisk(), lastChecksum);
|
|
|
+ } finally {
|
|
|
+ lock.unlock();
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public void waitForMinLength(long minLength, long time, TimeUnit unit)
|
|
|
+ throws IOException {
|
|
|
+ long nanos = unit.toNanos(time);
|
|
|
+ lock.lock();
|
|
|
+ try {
|
|
|
+ while (bytesOnDisk < minLength) {
|
|
|
+ if (nanos <= 0L) {
|
|
|
+ throw new IOException(
|
|
|
+ String.format("Need %d bytes, but only %d bytes available",
|
|
|
+ minLength, bytesOnDisk));
|
|
|
+ }
|
|
|
+ nanos = bytesOnDiskChange.awaitNanos(nanos);
|
|
|
+ }
|
|
|
+ } catch (InterruptedException e) {
|
|
|
+ throw new IOException(e);
|
|
|
+ } finally {
|
|
|
+ lock.unlock();
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
@Override // ReplicaInPipeline
|