|
@@ -47,12 +47,13 @@ class DFSClient implements FSConstants {
|
|
|
Random r = new Random();
|
|
|
String clientName;
|
|
|
Daemon leaseChecker;
|
|
|
-
|
|
|
+ private Configuration conf;
|
|
|
|
|
|
/**
|
|
|
* Create a new DFSClient connected to the given namenode server.
|
|
|
*/
|
|
|
public DFSClient(InetSocketAddress nameNodeAddr, Configuration conf) {
|
|
|
+ this.conf = conf;
|
|
|
this.namenode = (ClientProtocol) RPC.getProxy(ClientProtocol.class, nameNodeAddr, conf);
|
|
|
try {
|
|
|
this.localName = InetAddress.getLocalHost().getHostName();
|
|
@@ -255,6 +256,7 @@ class DFSClient implements FSConstants {
|
|
|
* negotiation of the namenode and various datanodes as necessary.
|
|
|
****************************************************************/
|
|
|
class DFSInputStream extends FSInputStream {
|
|
|
+ private Socket s = null;
|
|
|
boolean closed = false;
|
|
|
|
|
|
private String src;
|
|
@@ -316,9 +318,9 @@ class DFSClient implements FSConstants {
|
|
|
throw new IOException("Attempted to read past end of file");
|
|
|
}
|
|
|
|
|
|
- if (blockStream != null) {
|
|
|
- blockStream.close();
|
|
|
- partnerStream.close();
|
|
|
+ if (s != null) {
|
|
|
+ s.close();
|
|
|
+ s = null;
|
|
|
}
|
|
|
|
|
|
//
|
|
@@ -348,7 +350,6 @@ class DFSClient implements FSConstants {
|
|
|
//
|
|
|
int failures = 0;
|
|
|
InetSocketAddress targetAddr = null;
|
|
|
- Socket s = null;
|
|
|
TreeSet deadNodes = new TreeSet();
|
|
|
while (s == null) {
|
|
|
DatanodeInfo chosenNode;
|
|
@@ -376,8 +377,9 @@ class DFSClient implements FSConstants {
|
|
|
continue;
|
|
|
}
|
|
|
try {
|
|
|
- s = new Socket(targetAddr.getAddress(), targetAddr.getPort());
|
|
|
- //s.setSoTimeout(READ_TIMEOUT);
|
|
|
+ s = new Socket();
|
|
|
+ s.connect(targetAddr, READ_TIMEOUT);
|
|
|
+ s.setSoTimeout(READ_TIMEOUT);
|
|
|
|
|
|
//
|
|
|
// Xmit header info to datanode
|
|
@@ -428,10 +430,10 @@ class DFSClient implements FSConstants {
|
|
|
throw new IOException("Stream closed");
|
|
|
}
|
|
|
|
|
|
- if (blockStream != null) {
|
|
|
+ if (s != null) {
|
|
|
blockStream.close();
|
|
|
- blockStream = null;
|
|
|
- partnerStream.close();
|
|
|
+ s.close();
|
|
|
+ s = null;
|
|
|
}
|
|
|
super.close();
|
|
|
closed = true;
|
|
@@ -520,6 +522,7 @@ class DFSClient implements FSConstants {
|
|
|
* DFSOutputStream creates files from a stream of bytes.
|
|
|
****************************************************************/
|
|
|
class DFSOutputStream extends FSOutputStream {
|
|
|
+ private Socket s;
|
|
|
boolean closed = false;
|
|
|
|
|
|
private byte outBuf[] = new byte[BUFFER_SIZE];
|
|
@@ -528,8 +531,11 @@ class DFSClient implements FSConstants {
|
|
|
private UTF8 src;
|
|
|
boolean closingDown = false;
|
|
|
private boolean overwrite;
|
|
|
+ private boolean blockStreamWorking;
|
|
|
private DataOutputStream blockStream;
|
|
|
private DataInputStream blockReplyStream;
|
|
|
+ private File backupFile;
|
|
|
+ private OutputStream backupStream;
|
|
|
private Block block;
|
|
|
private DatanodeInfo targets[];
|
|
|
private long filePos = 0;
|
|
@@ -543,16 +549,31 @@ class DFSClient implements FSConstants {
|
|
|
this.overwrite = overwrite;
|
|
|
this.blockStream = null;
|
|
|
this.blockReplyStream = null;
|
|
|
+ this.blockStreamWorking = false;
|
|
|
+ this.backupFile = newBackupFile();
|
|
|
|
|
|
+ this.backupStream = new BufferedOutputStream(new FileOutputStream(backupFile));
|
|
|
nextBlockOutputStream(true);
|
|
|
}
|
|
|
|
|
|
+ private File newBackupFile() throws IOException {
|
|
|
+ return conf.getFile("dfs.data.dir",
|
|
|
+ "tmp"+File.separator+
|
|
|
+ "client-"+Math.abs(r.nextLong()));
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
* Open a DataOutputStream to a DataNode so that it can be written to.
|
|
|
* This happens when a file is created and each time a new block is allocated.
|
|
|
* Must get block ID and the IDs of the destinations from the namenode.
|
|
|
*/
|
|
|
private synchronized void nextBlockOutputStream(boolean firstTime) throws IOException {
|
|
|
+ if (! firstTime && blockStreamWorking) {
|
|
|
+ blockStream.flush();
|
|
|
+ s.close();
|
|
|
+ blockStreamWorking = false;
|
|
|
+ }
|
|
|
+
|
|
|
boolean retry = false;
|
|
|
long start = System.currentTimeMillis();
|
|
|
do {
|
|
@@ -588,10 +609,10 @@ class DFSClient implements FSConstants {
|
|
|
// Connect to first DataNode in the list. Abort if this fails.
|
|
|
//
|
|
|
InetSocketAddress target = DataNode.createSocketAddr(nodes[0].getName().toString());
|
|
|
- Socket s = null;
|
|
|
try {
|
|
|
- s = new Socket(target.getAddress(), target.getPort());
|
|
|
- //s.setSoTimeout(READ_TIMEOUT);
|
|
|
+ s = new Socket();
|
|
|
+ s.connect(target, READ_TIMEOUT);
|
|
|
+ s.setSoTimeout(READ_TIMEOUT);
|
|
|
} catch (IOException ie) {
|
|
|
// Connection failed. Let's wait a little bit and retry
|
|
|
try {
|
|
@@ -625,6 +646,7 @@ class DFSClient implements FSConstants {
|
|
|
bytesWrittenToBlock = 0;
|
|
|
blockStream = out;
|
|
|
blockReplyStream = new DataInputStream(new BufferedInputStream(s.getInputStream()));
|
|
|
+ blockStreamWorking = true;
|
|
|
} while (retry);
|
|
|
}
|
|
|
|
|
@@ -705,21 +727,18 @@ class DFSClient implements FSConstants {
|
|
|
//
|
|
|
// To the blockStream, write length, then bytes
|
|
|
//
|
|
|
- try {
|
|
|
- blockStream.writeLong(workingPos);
|
|
|
- blockStream.write(outBuf, 0, workingPos);
|
|
|
- } catch (IOException ie) {
|
|
|
- try {
|
|
|
- blockStream.close();
|
|
|
- } catch (IOException ie2) {
|
|
|
- }
|
|
|
- try {
|
|
|
- blockReplyStream.close();
|
|
|
- } catch (IOException ie2) {
|
|
|
- }
|
|
|
- namenode.abandonBlock(block, src.toString());
|
|
|
- throw ie;
|
|
|
+ if (blockStreamWorking) {
|
|
|
+ try {
|
|
|
+ blockStream.writeLong(workingPos);
|
|
|
+ blockStream.write(outBuf, 0, workingPos);
|
|
|
+ } catch (IOException ie) {
|
|
|
+ handleSocketException(ie);
|
|
|
+ }
|
|
|
}
|
|
|
+ //
|
|
|
+ // To the local block backup, write just the bytes
|
|
|
+ //
|
|
|
+ backupStream.write(outBuf, 0, workingPos);
|
|
|
|
|
|
//
|
|
|
// Track position
|
|
@@ -734,20 +753,64 @@ class DFSClient implements FSConstants {
|
|
|
* We're done writing to the current block.
|
|
|
*/
|
|
|
private synchronized void endBlock() throws IOException {
|
|
|
- try {
|
|
|
- internalClose();
|
|
|
- } catch (IOException ie) {
|
|
|
- namenode.abandonBlock(block, src.toString());
|
|
|
- throw ie;
|
|
|
+ boolean mustRecover = ! blockStreamWorking;
|
|
|
+
|
|
|
+ //
|
|
|
+ // A zero-length set of data indicates the end of the block
|
|
|
+ //
|
|
|
+ if (blockStreamWorking) {
|
|
|
+ try {
|
|
|
+ internalClose();
|
|
|
+ } catch (IOException ie) {
|
|
|
+ handleSocketException(ie);
|
|
|
+ mustRecover = true;
|
|
|
+ } finally {
|
|
|
+ blockStreamWorking = false;
|
|
|
+ }
|
|
|
}
|
|
|
+
|
|
|
+ //
|
|
|
+ // Done with local copy
|
|
|
+ //
|
|
|
+ backupStream.close();
|
|
|
+
|
|
|
+ //
|
|
|
+ // If necessary, recover from a failed datanode connection.
|
|
|
+ //
|
|
|
+ while (mustRecover) {
|
|
|
+ nextBlockOutputStream(false);
|
|
|
+ InputStream in = new FileInputStream(backupFile);
|
|
|
+ try {
|
|
|
+ byte buf[] = new byte[BUFFER_SIZE];
|
|
|
+ int bytesRead = in.read(buf);
|
|
|
+ while (bytesRead >= 0) {
|
|
|
+ blockStream.writeLong((long) bytesRead);
|
|
|
+ blockStream.write(buf, 0, bytesRead);
|
|
|
+ bytesRead = in.read(buf);
|
|
|
+ }
|
|
|
+ internalClose();
|
|
|
+ LOG.info("Recovered from failed datanode connection");
|
|
|
+ mustRecover = false;
|
|
|
+ } catch (IOException ie) {
|
|
|
+ handleSocketException(ie);
|
|
|
+ } finally {
|
|
|
+ in.close();
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ //
|
|
|
+ // Delete local backup, start new one
|
|
|
+ //
|
|
|
+ backupFile.delete();
|
|
|
+ backupFile = newBackupFile();
|
|
|
+ backupStream = new BufferedOutputStream(new FileOutputStream(backupFile));
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
- * Close down stream to remote datanode.
|
|
|
+ * Close down stream to remote datanode. Called from two places
|
|
|
+ * in endBlock();
|
|
|
*/
|
|
|
private synchronized void internalClose() throws IOException {
|
|
|
- try {
|
|
|
- // A zero-length set of data indicates the end of the block
|
|
|
blockStream.writeLong(0);
|
|
|
blockStream.flush();
|
|
|
|
|
@@ -761,16 +824,18 @@ class DFSClient implements FSConstants {
|
|
|
lb.readFields(blockReplyStream);
|
|
|
namenode.reportWrittenBlock(lb);
|
|
|
|
|
|
- } finally {
|
|
|
- try {
|
|
|
- blockStream.close();
|
|
|
- } catch (IOException ie2) {
|
|
|
- }
|
|
|
- try {
|
|
|
- blockReplyStream.close();
|
|
|
- } catch (IOException ie2) {
|
|
|
- }
|
|
|
+ s.close();
|
|
|
+ }
|
|
|
+
|
|
|
+ private void handleSocketException(IOException ie) throws IOException {
|
|
|
+ LOG.log(Level.WARNING, "Error while writing.", ie);
|
|
|
+ try {
|
|
|
+ s.close();
|
|
|
+ } catch (IOException ie2) {
|
|
|
+ LOG.log(Level.WARNING, "Error closing socket.", ie2);
|
|
|
}
|
|
|
+ blockStreamWorking = false;
|
|
|
+ namenode.abandonBlock(block, src.toString());
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -786,9 +851,13 @@ class DFSClient implements FSConstants {
|
|
|
flush();
|
|
|
endBlock();
|
|
|
|
|
|
- blockStream.close();
|
|
|
- blockReplyStream.close();
|
|
|
+ backupStream.close();
|
|
|
+ backupFile.delete();
|
|
|
|
|
|
+ if (blockStreamWorking) {
|
|
|
+ s.close();
|
|
|
+ blockStreamWorking = false;
|
|
|
+ }
|
|
|
super.close();
|
|
|
|
|
|
long localstart = System.currentTimeMillis();
|