|
@@ -18,8 +18,6 @@
|
|
|
package org.apache.hadoop.hdfs.nfs.nfs3;
|
|
|
|
|
|
import java.io.IOException;
|
|
|
-import java.util.Iterator;
|
|
|
-import java.util.Map.Entry;
|
|
|
import java.util.concurrent.ConcurrentMap;
|
|
|
|
|
|
import org.apache.commons.logging.Log;
|
|
@@ -29,11 +27,12 @@ import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
|
|
|
import org.apache.hadoop.hdfs.DFSClient;
|
|
|
import org.apache.hadoop.hdfs.client.HdfsDataOutputStream;
|
|
|
import org.apache.hadoop.hdfs.nfs.nfs3.OpenFileCtx.COMMIT_STATUS;
|
|
|
+import org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException;
|
|
|
+import org.apache.hadoop.ipc.RemoteException;
|
|
|
import org.apache.hadoop.nfs.NfsFileType;
|
|
|
import org.apache.hadoop.nfs.nfs3.FileHandle;
|
|
|
import org.apache.hadoop.nfs.nfs3.IdUserGroup;
|
|
|
import org.apache.hadoop.nfs.nfs3.Nfs3Constant;
|
|
|
-import org.apache.hadoop.nfs.nfs3.Nfs3Constant.WriteStableHow;
|
|
|
import org.apache.hadoop.nfs.nfs3.Nfs3FileAttributes;
|
|
|
import org.apache.hadoop.nfs.nfs3.Nfs3Status;
|
|
|
import org.apache.hadoop.nfs.nfs3.request.WRITE3Request;
|
|
@@ -56,69 +55,70 @@ public class WriteManager {
|
|
|
|
|
|
private final Configuration config;
|
|
|
private final IdUserGroup iug;
|
|
|
- private final ConcurrentMap<FileHandle, OpenFileCtx> openFileMap = Maps
|
|
|
- .newConcurrentMap();
|
|
|
-
|
|
|
+
|
|
|
private AsyncDataService asyncDataService;
|
|
|
private boolean asyncDataServiceStarted = false;
|
|
|
|
|
|
- private final StreamMonitor streamMonitor;
|
|
|
-
|
|
|
+ private final int maxStreams;
|
|
|
+
|
|
|
/**
|
|
|
* The time limit to wait for accumulate reordered sequential writes to the
|
|
|
* same file before the write is considered done.
|
|
|
*/
|
|
|
private long streamTimeout;
|
|
|
-
|
|
|
- public static final long DEFAULT_STREAM_TIMEOUT = 10 * 60 * 1000; //10 minutes
|
|
|
- public static final long MINIMIUM_STREAM_TIMEOUT = 10 * 1000; //10 seconds
|
|
|
-
|
|
|
- void addOpenFileStream(FileHandle h, OpenFileCtx ctx) {
|
|
|
- openFileMap.put(h, ctx);
|
|
|
- if (LOG.isDebugEnabled()) {
|
|
|
- LOG.debug("After add the new stream " + h.getFileId()
|
|
|
- + ", the stream number:" + openFileMap.size());
|
|
|
+
|
|
|
+ private final OpenFileCtxCache fileContextCache;
|
|
|
+
|
|
|
+ static public class MultipleCachedStreamException extends IOException {
|
|
|
+ private static final long serialVersionUID = 1L;
|
|
|
+
|
|
|
+ public MultipleCachedStreamException(String msg) {
|
|
|
+ super(msg);
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ boolean addOpenFileStream(FileHandle h, OpenFileCtx ctx) {
|
|
|
+ return fileContextCache.put(h, ctx);
|
|
|
+ }
|
|
|
+
|
|
|
WriteManager(IdUserGroup iug, final Configuration config) {
|
|
|
this.iug = iug;
|
|
|
this.config = config;
|
|
|
-
|
|
|
- streamTimeout = config.getLong("dfs.nfs3.stream.timeout",
|
|
|
- DEFAULT_STREAM_TIMEOUT);
|
|
|
+ streamTimeout = config.getLong(Nfs3Constant.OUTPUT_STREAM_TIMEOUT,
|
|
|
+ Nfs3Constant.OUTPUT_STREAM_TIMEOUT_DEFAULT);
|
|
|
LOG.info("Stream timeout is " + streamTimeout + "ms.");
|
|
|
- if (streamTimeout < MINIMIUM_STREAM_TIMEOUT) {
|
|
|
+ if (streamTimeout < Nfs3Constant.OUTPUT_STREAM_TIMEOUT_MIN_DEFAULT) {
|
|
|
LOG.info("Reset stream timeout to minimum value "
|
|
|
- + MINIMIUM_STREAM_TIMEOUT + "ms.");
|
|
|
- streamTimeout = MINIMIUM_STREAM_TIMEOUT;
|
|
|
+ + Nfs3Constant.OUTPUT_STREAM_TIMEOUT_MIN_DEFAULT + "ms.");
|
|
|
+ streamTimeout = Nfs3Constant.OUTPUT_STREAM_TIMEOUT_MIN_DEFAULT;
|
|
|
}
|
|
|
-
|
|
|
- this.streamMonitor = new StreamMonitor();
|
|
|
+ maxStreams = config.getInt(Nfs3Constant.MAX_OPEN_FILES,
|
|
|
+ Nfs3Constant.MAX_OPEN_FILES_DEFAULT);
|
|
|
+ LOG.info("Maximum open streams is "+ maxStreams);
|
|
|
+ this.fileContextCache = new OpenFileCtxCache(config, streamTimeout);
|
|
|
}
|
|
|
|
|
|
- private void startAsyncDataSerivce() {
|
|
|
- streamMonitor.start();
|
|
|
+ void startAsyncDataSerivce() {
|
|
|
+ if (asyncDataServiceStarted) {
|
|
|
+ return;
|
|
|
+ }
|
|
|
+ fileContextCache.start();
|
|
|
this.asyncDataService = new AsyncDataService();
|
|
|
asyncDataServiceStarted = true;
|
|
|
}
|
|
|
|
|
|
- private void shutdownAsyncDataService() {
|
|
|
- asyncDataService.shutdown();
|
|
|
+ void shutdownAsyncDataService() {
|
|
|
+ if (!asyncDataServiceStarted) {
|
|
|
+ return;
|
|
|
+ }
|
|
|
asyncDataServiceStarted = false;
|
|
|
- streamMonitor.interrupt();
|
|
|
+ asyncDataService.shutdown();
|
|
|
+ fileContextCache.shutdown();
|
|
|
}
|
|
|
|
|
|
void handleWrite(DFSClient dfsClient, WRITE3Request request, Channel channel,
|
|
|
int xid, Nfs3FileAttributes preOpAttr) throws IOException {
|
|
|
- // First write request starts the async data service
|
|
|
- if (!asyncDataServiceStarted) {
|
|
|
- startAsyncDataSerivce();
|
|
|
- }
|
|
|
-
|
|
|
- long offset = request.getOffset();
|
|
|
int count = request.getCount();
|
|
|
- WriteStableHow stableHow = request.getStableHow();
|
|
|
byte[] data = request.getData().array();
|
|
|
if (data.length < count) {
|
|
|
WRITE3Response response = new WRITE3Response(Nfs3Status.NFS3ERR_INVAL);
|
|
@@ -129,13 +129,12 @@ public class WriteManager {
|
|
|
|
|
|
FileHandle handle = request.getHandle();
|
|
|
if (LOG.isDebugEnabled()) {
|
|
|
- LOG.debug("handleWrite fileId: " + handle.getFileId() + " offset: "
|
|
|
- + offset + " length:" + count + " stableHow:" + stableHow.getValue());
|
|
|
+ LOG.debug("handleWrite " + request);
|
|
|
}
|
|
|
|
|
|
// Check if there is a stream to write
|
|
|
FileHandle fileHandle = request.getHandle();
|
|
|
- OpenFileCtx openFileCtx = openFileMap.get(fileHandle);
|
|
|
+ OpenFileCtx openFileCtx = fileContextCache.get(fileHandle);
|
|
|
if (openFileCtx == null) {
|
|
|
LOG.info("No opened stream for fileId:" + fileHandle.getFileId());
|
|
|
|
|
@@ -150,6 +149,15 @@ public class WriteManager {
|
|
|
fos = dfsClient.append(fileIdPath, bufferSize, null, null);
|
|
|
|
|
|
latestAttr = Nfs3Utils.getFileAttr(dfsClient, fileIdPath, iug);
|
|
|
+ } catch (RemoteException e) {
|
|
|
+ IOException io = e.unwrapRemoteException();
|
|
|
+ if (io instanceof AlreadyBeingCreatedException) {
|
|
|
+ LOG.warn("Can't append file:" + fileIdPath
|
|
|
+ + ". Possibly the file is being closed. Drop the request:"
|
|
|
+ + request + ", wait for the client to retry...");
|
|
|
+ return;
|
|
|
+ }
|
|
|
+ throw e;
|
|
|
} catch (IOException e) {
|
|
|
LOG.error("Can't apapend to file:" + fileIdPath + ", error:" + e);
|
|
|
if (fos != null) {
|
|
@@ -170,9 +178,26 @@ public class WriteManager {
|
|
|
Nfs3Constant.FILE_DUMP_DIR_DEFAULT);
|
|
|
openFileCtx = new OpenFileCtx(fos, latestAttr, writeDumpDir + "/"
|
|
|
+ fileHandle.getFileId(), dfsClient, iug);
|
|
|
- addOpenFileStream(fileHandle, openFileCtx);
|
|
|
+
|
|
|
+ if (!addOpenFileStream(fileHandle, openFileCtx)) {
|
|
|
+ LOG.info("Can't add new stream. Close it. Tell client to retry.");
|
|
|
+ try {
|
|
|
+ fos.close();
|
|
|
+ } catch (IOException e) {
|
|
|
+ LOG.error("Can't close stream for fileId:" + handle.getFileId());
|
|
|
+ }
|
|
|
+ // Notify client to retry
|
|
|
+ WccData fileWcc = new WccData(latestAttr.getWccAttr(), latestAttr);
|
|
|
+ WRITE3Response response = new WRITE3Response(Nfs3Status.NFS3ERR_JUKEBOX,
|
|
|
+ fileWcc, 0, request.getStableHow(), Nfs3Constant.WRITE_COMMIT_VERF);
|
|
|
+ Nfs3Utils.writeChannel(channel,
|
|
|
+ response.writeHeaderAndResponse(new XDR(), xid, new VerifierNone()),
|
|
|
+ xid);
|
|
|
+ return;
|
|
|
+ }
|
|
|
+
|
|
|
if (LOG.isDebugEnabled()) {
|
|
|
- LOG.debug("opened stream for file:" + fileHandle.getFileId());
|
|
|
+ LOG.debug("Opened stream for appending file:" + fileHandle.getFileId());
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -185,7 +210,7 @@ public class WriteManager {
|
|
|
void handleCommit(DFSClient dfsClient, FileHandle fileHandle,
|
|
|
long commitOffset, Channel channel, int xid, Nfs3FileAttributes preOpAttr) {
|
|
|
int status;
|
|
|
- OpenFileCtx openFileCtx = openFileMap.get(fileHandle);
|
|
|
+ OpenFileCtx openFileCtx = fileContextCache.get(fileHandle);
|
|
|
|
|
|
if (openFileCtx == null) {
|
|
|
LOG.info("No opened stream for fileId:" + fileHandle.getFileId()
|
|
@@ -238,7 +263,7 @@ public class WriteManager {
|
|
|
String fileIdPath = Nfs3Utils.getFileIdPath(fileHandle);
|
|
|
Nfs3FileAttributes attr = Nfs3Utils.getFileAttr(client, fileIdPath, iug);
|
|
|
if (attr != null) {
|
|
|
- OpenFileCtx openFileCtx = openFileMap.get(fileHandle);
|
|
|
+ OpenFileCtx openFileCtx = fileContextCache.get(fileHandle);
|
|
|
if (openFileCtx != null) {
|
|
|
attr.setSize(openFileCtx.getNextOffset());
|
|
|
attr.setUsed(openFileCtx.getNextOffset());
|
|
@@ -253,8 +278,8 @@ public class WriteManager {
|
|
|
Nfs3FileAttributes attr = Nfs3Utils.getFileAttr(client, fileIdPath, iug);
|
|
|
|
|
|
if ((attr != null) && (attr.getType() == NfsFileType.NFSREG.toValue())) {
|
|
|
- OpenFileCtx openFileCtx = openFileMap
|
|
|
- .get(new FileHandle(attr.getFileId()));
|
|
|
+ OpenFileCtx openFileCtx = fileContextCache.get(new FileHandle(attr
|
|
|
+ .getFileId()));
|
|
|
|
|
|
if (openFileCtx != null) {
|
|
|
attr.setSize(openFileCtx.getNextOffset());
|
|
@@ -263,56 +288,9 @@ public class WriteManager {
|
|
|
}
|
|
|
return attr;
|
|
|
}
|
|
|
-
|
|
|
- @VisibleForTesting
|
|
|
- ConcurrentMap<FileHandle, OpenFileCtx> getOpenFileMap() {
|
|
|
- return this.openFileMap;
|
|
|
- }
|
|
|
-
|
|
|
- /**
|
|
|
- * StreamMonitor wakes up periodically to find and closes idle streams.
|
|
|
- */
|
|
|
- class StreamMonitor extends Daemon {
|
|
|
- private int rotation = 5 * 1000; // 5 seconds
|
|
|
- private long lastWakeupTime = 0;
|
|
|
-
|
|
|
- @Override
|
|
|
- public void run() {
|
|
|
- while (true) {
|
|
|
- Iterator<Entry<FileHandle, OpenFileCtx>> it = openFileMap.entrySet()
|
|
|
- .iterator();
|
|
|
- if (LOG.isTraceEnabled()) {
|
|
|
- LOG.trace("openFileMap size:" + openFileMap.size());
|
|
|
- }
|
|
|
- while (it.hasNext()) {
|
|
|
- Entry<FileHandle, OpenFileCtx> pairs = it.next();
|
|
|
- OpenFileCtx ctx = pairs.getValue();
|
|
|
- if (ctx.streamCleanup((pairs.getKey()).getFileId(), streamTimeout)) {
|
|
|
- it.remove();
|
|
|
- if (LOG.isDebugEnabled()) {
|
|
|
- LOG.debug("After remove stream " + pairs.getKey().getFileId()
|
|
|
- + ", the stream number:" + openFileMap.size());
|
|
|
- }
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- // Check if it can sleep
|
|
|
- try {
|
|
|
- long workedTime = System.currentTimeMillis() - lastWakeupTime;
|
|
|
- if (workedTime < rotation) {
|
|
|
- if (LOG.isTraceEnabled()) {
|
|
|
- LOG.trace("StreamMonitor can still have a sleep:"
|
|
|
- + ((rotation - workedTime) / 1000));
|
|
|
- }
|
|
|
- Thread.sleep(rotation - workedTime);
|
|
|
- }
|
|
|
- lastWakeupTime = System.currentTimeMillis();
|
|
|
|
|
|
- } catch (InterruptedException e) {
|
|
|
- LOG.info("StreamMonitor got interrupted");
|
|
|
- return;
|
|
|
- }
|
|
|
- }
|
|
|
- }
|
|
|
+ @VisibleForTesting
|
|
|
+ OpenFileCtxCache getOpenFileCtxCache() {
|
|
|
+ return this.fileContextCache;
|
|
|
}
|
|
|
}
|