|
@@ -75,6 +75,8 @@ import org.slf4j.LoggerFactory;
|
|
import com.google.common.annotations.VisibleForTesting;
|
|
import com.google.common.annotations.VisibleForTesting;
|
|
import com.google.common.base.Preconditions;
|
|
import com.google.common.base.Preconditions;
|
|
|
|
|
|
|
|
+import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.Write.RECOVER_LEASE_ON_CLOSE_EXCEPTION_DEFAULT;
|
|
|
|
+import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.Write.RECOVER_LEASE_ON_CLOSE_EXCEPTION_KEY;
|
|
|
|
|
|
/****************************************************************
|
|
/****************************************************************
|
|
* DFSOutputStream creates files from a stream of bytes.
|
|
* DFSOutputStream creates files from a stream of bytes.
|
|
@@ -126,6 +128,7 @@ public class DFSOutputStream extends FSOutputSummer
|
|
protected final AtomicReference<CachingStrategy> cachingStrategy;
|
|
protected final AtomicReference<CachingStrategy> cachingStrategy;
|
|
private FileEncryptionInfo fileEncryptionInfo;
|
|
private FileEncryptionInfo fileEncryptionInfo;
|
|
private int writePacketSize;
|
|
private int writePacketSize;
|
|
|
|
+ private boolean leaseRecovered = false;
|
|
|
|
|
|
/** Use {@link ByteArrayManager} to create buffer for non-heartbeat packets.*/
|
|
/** Use {@link ByteArrayManager} to create buffer for non-heartbeat packets.*/
|
|
protected DFSPacket createPacket(int packetSize, int chunksPerPkt,
|
|
protected DFSPacket createPacket(int packetSize, int chunksPerPkt,
|
|
@@ -861,7 +864,14 @@ public class DFSOutputStream extends FSOutputSummer
|
|
}
|
|
}
|
|
|
|
|
|
protected synchronized void closeImpl() throws IOException {
|
|
protected synchronized void closeImpl() throws IOException {
|
|
|
|
+ boolean recoverLeaseOnCloseException = dfsClient.getConfiguration()
|
|
|
|
+ .getBoolean(RECOVER_LEASE_ON_CLOSE_EXCEPTION_KEY,
|
|
|
|
+ RECOVER_LEASE_ON_CLOSE_EXCEPTION_DEFAULT);
|
|
if (isClosed()) {
|
|
if (isClosed()) {
|
|
|
|
+ if (!leaseRecovered) {
|
|
|
|
+ recoverLease(recoverLeaseOnCloseException);
|
|
|
|
+ }
|
|
|
|
+
|
|
LOG.debug("Closing an already closed stream. [Stream:{}, streamer:{}]",
|
|
LOG.debug("Closing an already closed stream. [Stream:{}, streamer:{}]",
|
|
closed, getStreamer().streamerClosed());
|
|
closed, getStreamer().streamerClosed());
|
|
try {
|
|
try {
|
|
@@ -896,6 +906,9 @@ public class DFSOutputStream extends FSOutputSummer
|
|
}
|
|
}
|
|
completeFile();
|
|
completeFile();
|
|
} catch (ClosedChannelException ignored) {
|
|
} catch (ClosedChannelException ignored) {
|
|
|
|
+ } catch (IOException ioe) {
|
|
|
|
+ recoverLease(recoverLeaseOnCloseException);
|
|
|
|
+ throw ioe;
|
|
} finally {
|
|
} finally {
|
|
// Failures may happen when flushing data.
|
|
// Failures may happen when flushing data.
|
|
// Streamers may keep waiting for the new block information.
|
|
// Streamers may keep waiting for the new block information.
|
|
@@ -906,7 +919,23 @@ public class DFSOutputStream extends FSOutputSummer
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
- private void completeFile() throws IOException {
|
|
|
|
|
|
+ /**
|
|
|
|
+ * If recoverLeaseOnCloseException is true and an exception occurs when
|
|
|
|
+ * closing a file, recover lease.
|
|
|
|
+ */
|
|
|
|
+ private void recoverLease(boolean recoverLeaseOnCloseException) {
|
|
|
|
+ if (recoverLeaseOnCloseException) {
|
|
|
|
+ try {
|
|
|
|
+ dfsClient.endFileLease(fileId);
|
|
|
|
+ dfsClient.recoverLease(src);
|
|
|
|
+ leaseRecovered = true;
|
|
|
|
+ } catch (Exception e) {
|
|
|
|
+ LOG.warn("Fail to recover lease for {}", src, e);
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ void completeFile() throws IOException {
|
|
// get last block before destroying the streamer
|
|
// get last block before destroying the streamer
|
|
ExtendedBlock lastBlock = getStreamer().getBlock();
|
|
ExtendedBlock lastBlock = getStreamer().getBlock();
|
|
try (TraceScope ignored =
|
|
try (TraceScope ignored =
|
|
@@ -1076,6 +1105,11 @@ public class DFSOutputStream extends FSOutputSummer
|
|
return getClass().getSimpleName() + ":" + streamer;
|
|
return getClass().getSimpleName() + ":" + streamer;
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ @VisibleForTesting
|
|
|
|
+ boolean isLeaseRecovered() {
|
|
|
|
+ return leaseRecovered;
|
|
|
|
+ }
|
|
|
|
+
|
|
static LocatedBlock addBlock(DatanodeInfo[] excludedNodes,
|
|
static LocatedBlock addBlock(DatanodeInfo[] excludedNodes,
|
|
DFSClient dfsClient, String src, ExtendedBlock prevBlock, long fileId,
|
|
DFSClient dfsClient, String src, ExtendedBlock prevBlock, long fileId,
|
|
String[] favoredNodes, EnumSet<AddBlockFlag> allocFlags)
|
|
String[] favoredNodes, EnumSet<AddBlockFlag> allocFlags)
|