|
@@ -18,7 +18,6 @@
|
|
|
|
|
|
package org.apache.hadoop.hdfs.server.datanode;
|
|
|
|
|
|
-
|
|
|
import org.apache.commons.io.FileUtils;
|
|
|
import org.apache.hadoop.classification.InterfaceAudience;
|
|
|
import org.apache.hadoop.classification.InterfaceStability;
|
|
@@ -34,7 +33,6 @@ import org.apache.hadoop.io.LongWritable;
|
|
|
import org.apache.hadoop.io.nativeio.NativeIO;
|
|
|
import org.apache.hadoop.io.nativeio.NativeIOException;
|
|
|
import org.apache.hadoop.net.SocketOutputStream;
|
|
|
-import org.apache.hadoop.util.ReflectionUtils;
|
|
|
import org.slf4j.Logger;
|
|
|
import org.slf4j.LoggerFactory;
|
|
|
|
|
@@ -59,11 +57,14 @@ import static org.apache.hadoop.hdfs.server.datanode.FileIoProvider.OPERATION.*;
|
|
|
|
|
|
/**
|
|
|
* This class abstracts out various file IO operations performed by the
|
|
|
- * DataNode and invokes event hooks before and after each file IO.
|
|
|
+ * DataNode and invokes profiling (for collecting stats) and fault injection
|
|
|
+ * (for testing) event hooks before and after each file IO.
|
|
|
*
|
|
|
- * Behavior can be injected into these events by implementing
|
|
|
- * {@link FileIoEvents} and replacing the default implementation
|
|
|
- * with {@link DFSConfigKeys#DFS_DATANODE_FILE_IO_EVENTS_CLASS_KEY}.
|
|
|
+ * Behavior can be injected into these events by enabling the
|
|
|
+ * profiling and/or fault injection event hooks through
|
|
|
+ * {@link DFSConfigKeys#DFS_DATANODE_ENABLE_FILEIO_PROFILING_KEY} and
|
|
|
+ * {@link DFSConfigKeys#DFS_DATANODE_ENABLE_FILEIO_FAULT_INJECTION_KEY}.
|
|
|
+ * These event hooks are disabled by default.
|
|
|
*
|
|
|
* Most functions accept an optional {@link FsVolumeSpi} parameter for
|
|
|
* instrumentation/logging.
|
|
@@ -78,9 +79,12 @@ public class FileIoProvider {
|
|
|
public static final Logger LOG = LoggerFactory.getLogger(
|
|
|
FileIoProvider.class);
|
|
|
|
|
|
- private final FileIoEvents eventHooks;
|
|
|
+ private final ProfilingFileIoEvents profilingEventHook;
|
|
|
+ private final FaultInjectorFileIoEvents faultInjectorEventHook;
|
|
|
private final DataNode datanode;
|
|
|
|
|
|
+ private static final int LEN_INT = 4;
|
|
|
+
|
|
|
/**
|
|
|
* @param conf Configuration object. May be null. When null,
|
|
|
* the event handlers are no-ops.
|
|
@@ -89,15 +93,8 @@ public class FileIoProvider {
|
|
|
*/
|
|
|
public FileIoProvider(@Nullable Configuration conf,
|
|
|
final DataNode datanode) {
|
|
|
- if (conf != null) {
|
|
|
- final Class<? extends FileIoEvents> clazz = conf.getClass(
|
|
|
- DFSConfigKeys.DFS_DATANODE_FILE_IO_EVENTS_CLASS_KEY,
|
|
|
- DefaultFileIoEvents.class,
|
|
|
- FileIoEvents.class);
|
|
|
- eventHooks = ReflectionUtils.newInstance(clazz, conf);
|
|
|
- } else {
|
|
|
- eventHooks = new DefaultFileIoEvents();
|
|
|
- }
|
|
|
+ profilingEventHook = new ProfilingFileIoEvents(conf);
|
|
|
+ faultInjectorEventHook = new FaultInjectorFileIoEvents(conf);
|
|
|
this.datanode = datanode;
|
|
|
}
|
|
|
|
|
@@ -122,15 +119,6 @@ public class FileIoProvider {
|
|
|
NATIVE_COPY
|
|
|
}
|
|
|
|
|
|
- /**
|
|
|
- * Retrieve statistics from the underlying {@link FileIoEvents}
|
|
|
- * implementation as a JSON string, if it maintains them.
|
|
|
- * @return statistics as a JSON string. May be null.
|
|
|
- */
|
|
|
- public @Nullable String getStatistics() {
|
|
|
- return eventHooks.getStatistics();
|
|
|
- }
|
|
|
-
|
|
|
/**
|
|
|
* See {@link Flushable#flush()}.
|
|
|
*
|
|
@@ -139,12 +127,13 @@ public class FileIoProvider {
|
|
|
*/
|
|
|
public void flush(
|
|
|
@Nullable FsVolumeSpi volume, Flushable f) throws IOException {
|
|
|
- final long begin = eventHooks.beforeFileIo(volume, FLUSH, 0);
|
|
|
+ final long begin = profilingEventHook.beforeFileIo(volume, FLUSH, 0);
|
|
|
try {
|
|
|
+ faultInjectorEventHook.beforeFileIo(volume, FLUSH, 0);
|
|
|
f.flush();
|
|
|
- eventHooks.afterFileIo(volume, FLUSH, begin, 0);
|
|
|
+ profilingEventHook.afterFileIo(volume, FLUSH, begin, 0);
|
|
|
} catch (Exception e) {
|
|
|
- eventHooks.onFailure(datanode, volume, FLUSH, e, begin);
|
|
|
+ onFailure(volume, begin);
|
|
|
throw e;
|
|
|
}
|
|
|
}
|
|
@@ -157,12 +146,13 @@ public class FileIoProvider {
|
|
|
*/
|
|
|
public void sync(
|
|
|
@Nullable FsVolumeSpi volume, FileOutputStream fos) throws IOException {
|
|
|
- final long begin = eventHooks.beforeFileIo(volume, SYNC, 0);
|
|
|
+ final long begin = profilingEventHook.beforeFileIo(volume, SYNC, 0);
|
|
|
try {
|
|
|
+ faultInjectorEventHook.beforeFileIo(volume, SYNC, 0);
|
|
|
fos.getChannel().force(true);
|
|
|
- eventHooks.afterFileIo(volume, SYNC, begin, 0);
|
|
|
+ profilingEventHook.afterFileIo(volume, SYNC, begin, 0);
|
|
|
} catch (Exception e) {
|
|
|
- eventHooks.onFailure(datanode, volume, SYNC, e, begin);
|
|
|
+ onFailure(volume, begin);
|
|
|
throw e;
|
|
|
}
|
|
|
}
|
|
@@ -176,12 +166,13 @@ public class FileIoProvider {
|
|
|
public void syncFileRange(
|
|
|
@Nullable FsVolumeSpi volume, FileDescriptor outFd,
|
|
|
long offset, long numBytes, int flags) throws NativeIOException {
|
|
|
- final long begin = eventHooks.beforeFileIo(volume, SYNC, 0);
|
|
|
+ final long begin = profilingEventHook.beforeFileIo(volume, SYNC, 0);
|
|
|
try {
|
|
|
+ faultInjectorEventHook.beforeFileIo(volume, SYNC, 0);
|
|
|
NativeIO.POSIX.syncFileRangeIfPossible(outFd, offset, numBytes, flags);
|
|
|
- eventHooks.afterFileIo(volume, SYNC, begin, 0);
|
|
|
+ profilingEventHook.afterFileIo(volume, SYNC, begin, 0);
|
|
|
} catch (Exception e) {
|
|
|
- eventHooks.onFailure(datanode, volume, SYNC, e, begin);
|
|
|
+ onFailure(volume, begin);
|
|
|
throw e;
|
|
|
}
|
|
|
}
|
|
@@ -195,13 +186,14 @@ public class FileIoProvider {
|
|
|
public void posixFadvise(
|
|
|
@Nullable FsVolumeSpi volume, String identifier, FileDescriptor outFd,
|
|
|
long offset, long length, int flags) throws NativeIOException {
|
|
|
- final long begin = eventHooks.beforeMetadataOp(volume, FADVISE);
|
|
|
+ final long begin = profilingEventHook.beforeMetadataOp(volume, FADVISE);
|
|
|
try {
|
|
|
+ faultInjectorEventHook.beforeMetadataOp(volume, FADVISE);
|
|
|
NativeIO.POSIX.getCacheManipulator().posixFadviseIfPossible(
|
|
|
identifier, outFd, offset, length, flags);
|
|
|
- eventHooks.afterMetadataOp(volume, FADVISE, begin);
|
|
|
+ profilingEventHook.afterMetadataOp(volume, FADVISE, begin);
|
|
|
} catch (Exception e) {
|
|
|
- eventHooks.onFailure(datanode, volume, FADVISE, e, begin);
|
|
|
+ onFailure(volume, begin);
|
|
|
throw e;
|
|
|
}
|
|
|
}
|
|
@@ -213,13 +205,14 @@ public class FileIoProvider {
|
|
|
* @return true if the file was successfully deleted.
|
|
|
*/
|
|
|
public boolean delete(@Nullable FsVolumeSpi volume, File f) {
|
|
|
- final long begin = eventHooks.beforeMetadataOp(volume, DELETE);
|
|
|
+ final long begin = profilingEventHook.beforeMetadataOp(volume, DELETE);
|
|
|
try {
|
|
|
+ faultInjectorEventHook.beforeMetadataOp(volume, DELETE);
|
|
|
boolean deleted = f.delete();
|
|
|
- eventHooks.afterMetadataOp(volume, DELETE, begin);
|
|
|
+ profilingEventHook.afterMetadataOp(volume, DELETE, begin);
|
|
|
return deleted;
|
|
|
} catch (Exception e) {
|
|
|
- eventHooks.onFailure(datanode, volume, DELETE, e, begin);
|
|
|
+ onFailure(volume, begin);
|
|
|
throw e;
|
|
|
}
|
|
|
}
|
|
@@ -232,16 +225,17 @@ public class FileIoProvider {
|
|
|
* existed.
|
|
|
*/
|
|
|
public boolean deleteWithExistsCheck(@Nullable FsVolumeSpi volume, File f) {
|
|
|
- final long begin = eventHooks.beforeMetadataOp(volume, DELETE);
|
|
|
+ final long begin = profilingEventHook.beforeMetadataOp(volume, DELETE);
|
|
|
try {
|
|
|
+ faultInjectorEventHook.beforeMetadataOp(volume, DELETE);
|
|
|
boolean deleted = !f.exists() || f.delete();
|
|
|
- eventHooks.afterMetadataOp(volume, DELETE, begin);
|
|
|
+ profilingEventHook.afterMetadataOp(volume, DELETE, begin);
|
|
|
if (!deleted) {
|
|
|
LOG.warn("Failed to delete file {}", f);
|
|
|
}
|
|
|
return deleted;
|
|
|
} catch (Exception e) {
|
|
|
- eventHooks.onFailure(datanode, volume, DELETE, e, begin);
|
|
|
+ onFailure(volume, begin);
|
|
|
throw e;
|
|
|
}
|
|
|
}
|
|
@@ -263,13 +257,14 @@ public class FileIoProvider {
|
|
|
@Nullable FsVolumeSpi volume, SocketOutputStream sockOut,
|
|
|
FileChannel fileCh, long position, int count,
|
|
|
LongWritable waitTime, LongWritable transferTime) throws IOException {
|
|
|
- final long begin = eventHooks.beforeFileIo(volume, TRANSFER, count);
|
|
|
+ final long begin = profilingEventHook.beforeFileIo(volume, TRANSFER, count);
|
|
|
try {
|
|
|
+ faultInjectorEventHook.beforeFileIo(volume, TRANSFER, count);
|
|
|
sockOut.transferToFully(fileCh, position, count,
|
|
|
waitTime, transferTime);
|
|
|
- eventHooks.afterFileIo(volume, TRANSFER, begin, count);
|
|
|
+ profilingEventHook.afterFileIo(volume, TRANSFER, begin, count);
|
|
|
} catch (Exception e) {
|
|
|
- eventHooks.onFailure(datanode, volume, TRANSFER, e, begin);
|
|
|
+ onFailure(volume, begin);
|
|
|
throw e;
|
|
|
}
|
|
|
}
|
|
@@ -284,13 +279,14 @@ public class FileIoProvider {
|
|
|
*/
|
|
|
public boolean createFile(
|
|
|
@Nullable FsVolumeSpi volume, File f) throws IOException {
|
|
|
- final long begin = eventHooks.beforeMetadataOp(volume, OPEN);
|
|
|
+ final long begin = profilingEventHook.beforeMetadataOp(volume, OPEN);
|
|
|
try {
|
|
|
+ faultInjectorEventHook.beforeMetadataOp(volume, OPEN);
|
|
|
boolean created = f.createNewFile();
|
|
|
- eventHooks.afterMetadataOp(volume, OPEN, begin);
|
|
|
+ profilingEventHook.afterMetadataOp(volume, OPEN, begin);
|
|
|
return created;
|
|
|
} catch (Exception e) {
|
|
|
- eventHooks.onFailure(datanode, volume, OPEN, e, begin);
|
|
|
+ onFailure(volume, begin);
|
|
|
throw e;
|
|
|
}
|
|
|
}
|
|
@@ -309,15 +305,16 @@ public class FileIoProvider {
|
|
|
*/
|
|
|
public FileInputStream getFileInputStream(
|
|
|
@Nullable FsVolumeSpi volume, File f) throws FileNotFoundException {
|
|
|
- final long begin = eventHooks.beforeMetadataOp(volume, OPEN);
|
|
|
+ final long begin = profilingEventHook.beforeMetadataOp(volume, OPEN);
|
|
|
FileInputStream fis = null;
|
|
|
try {
|
|
|
+ faultInjectorEventHook.beforeMetadataOp(volume, OPEN);
|
|
|
fis = new WrappedFileInputStream(volume, f);
|
|
|
- eventHooks.afterMetadataOp(volume, OPEN, begin);
|
|
|
+ profilingEventHook.afterMetadataOp(volume, OPEN, begin);
|
|
|
return fis;
|
|
|
} catch(Exception e) {
|
|
|
org.apache.commons.io.IOUtils.closeQuietly(fis);
|
|
|
- eventHooks.onFailure(datanode, volume, OPEN, e, begin);
|
|
|
+ onFailure(volume, begin);
|
|
|
throw e;
|
|
|
}
|
|
|
}
|
|
@@ -339,15 +336,16 @@ public class FileIoProvider {
|
|
|
public FileOutputStream getFileOutputStream(
|
|
|
@Nullable FsVolumeSpi volume, File f,
|
|
|
boolean append) throws FileNotFoundException {
|
|
|
- final long begin = eventHooks.beforeMetadataOp(volume, OPEN);
|
|
|
+ final long begin = profilingEventHook.beforeMetadataOp(volume, OPEN);
|
|
|
FileOutputStream fos = null;
|
|
|
try {
|
|
|
+ faultInjectorEventHook.beforeMetadataOp(volume, OPEN);
|
|
|
fos = new WrappedFileOutputStream(volume, f, append);
|
|
|
- eventHooks.afterMetadataOp(volume, OPEN, begin);
|
|
|
+ profilingEventHook.afterMetadataOp(volume, OPEN, begin);
|
|
|
return fos;
|
|
|
} catch(Exception e) {
|
|
|
org.apache.commons.io.IOUtils.closeQuietly(fos);
|
|
|
- eventHooks.onFailure(datanode, volume, OPEN, e, begin);
|
|
|
+ onFailure(volume, begin);
|
|
|
throw e;
|
|
|
}
|
|
|
}
|
|
@@ -403,16 +401,17 @@ public class FileIoProvider {
|
|
|
public FileInputStream getShareDeleteFileInputStream(
|
|
|
@Nullable FsVolumeSpi volume, File f,
|
|
|
long offset) throws IOException {
|
|
|
- final long begin = eventHooks.beforeMetadataOp(volume, OPEN);
|
|
|
+ final long begin = profilingEventHook.beforeMetadataOp(volume, OPEN);
|
|
|
FileInputStream fis = null;
|
|
|
try {
|
|
|
+ faultInjectorEventHook.beforeMetadataOp(volume, OPEN);
|
|
|
fis = new WrappedFileInputStream(volume,
|
|
|
NativeIO.getShareDeleteFileDescriptor(f, offset));
|
|
|
- eventHooks.afterMetadataOp(volume, OPEN, begin);
|
|
|
+ profilingEventHook.afterMetadataOp(volume, OPEN, begin);
|
|
|
return fis;
|
|
|
} catch(Exception e) {
|
|
|
org.apache.commons.io.IOUtils.closeQuietly(fis);
|
|
|
- eventHooks.onFailure(datanode, volume, OPEN, e, begin);
|
|
|
+ onFailure(volume, begin);
|
|
|
throw e;
|
|
|
}
|
|
|
}
|
|
@@ -434,16 +433,17 @@ public class FileIoProvider {
|
|
|
*/
|
|
|
public FileInputStream openAndSeek(
|
|
|
@Nullable FsVolumeSpi volume, File f, long offset) throws IOException {
|
|
|
- final long begin = eventHooks.beforeMetadataOp(volume, OPEN);
|
|
|
+ final long begin = profilingEventHook.beforeMetadataOp(volume, OPEN);
|
|
|
FileInputStream fis = null;
|
|
|
try {
|
|
|
+ faultInjectorEventHook.beforeMetadataOp(volume, OPEN);
|
|
|
fis = new WrappedFileInputStream(volume,
|
|
|
FsDatasetUtil.openAndSeek(f, offset));
|
|
|
- eventHooks.afterMetadataOp(volume, OPEN, begin);
|
|
|
+ profilingEventHook.afterMetadataOp(volume, OPEN, begin);
|
|
|
return fis;
|
|
|
} catch(Exception e) {
|
|
|
org.apache.commons.io.IOUtils.closeQuietly(fis);
|
|
|
- eventHooks.onFailure(datanode, volume, OPEN, e, begin);
|
|
|
+ onFailure(volume, begin);
|
|
|
throw e;
|
|
|
}
|
|
|
}
|
|
@@ -465,15 +465,16 @@ public class FileIoProvider {
|
|
|
public RandomAccessFile getRandomAccessFile(
|
|
|
@Nullable FsVolumeSpi volume, File f,
|
|
|
String mode) throws FileNotFoundException {
|
|
|
- final long begin = eventHooks.beforeMetadataOp(volume, OPEN);
|
|
|
+ final long begin = profilingEventHook.beforeMetadataOp(volume, OPEN);
|
|
|
RandomAccessFile raf = null;
|
|
|
try {
|
|
|
+ faultInjectorEventHook.beforeMetadataOp(volume, OPEN);
|
|
|
raf = new WrappedRandomAccessFile(volume, f, mode);
|
|
|
- eventHooks.afterMetadataOp(volume, OPEN, begin);
|
|
|
+ profilingEventHook.afterMetadataOp(volume, OPEN, begin);
|
|
|
return raf;
|
|
|
} catch(Exception e) {
|
|
|
org.apache.commons.io.IOUtils.closeQuietly(raf);
|
|
|
- eventHooks.onFailure(datanode, volume, OPEN, e, begin);
|
|
|
+ onFailure(volume, begin);
|
|
|
throw e;
|
|
|
}
|
|
|
}
|
|
@@ -486,13 +487,14 @@ public class FileIoProvider {
|
|
|
* @return true on success false on failure.
|
|
|
*/
|
|
|
public boolean fullyDelete(@Nullable FsVolumeSpi volume, File dir) {
|
|
|
- final long begin = eventHooks.beforeMetadataOp(volume, DELETE);
|
|
|
+ final long begin = profilingEventHook.beforeMetadataOp(volume, DELETE);
|
|
|
try {
|
|
|
+ faultInjectorEventHook.beforeMetadataOp(volume, DELETE);
|
|
|
boolean deleted = FileUtil.fullyDelete(dir);
|
|
|
- eventHooks.afterMetadataOp(volume, DELETE, begin);
|
|
|
+ profilingEventHook.afterMetadataOp(volume, DELETE, begin);
|
|
|
return deleted;
|
|
|
} catch(Exception e) {
|
|
|
- eventHooks.onFailure(datanode, volume, DELETE, e, begin);
|
|
|
+ onFailure(volume, begin);
|
|
|
throw e;
|
|
|
}
|
|
|
}
|
|
@@ -508,12 +510,13 @@ public class FileIoProvider {
|
|
|
*/
|
|
|
public void replaceFile(
|
|
|
@Nullable FsVolumeSpi volume, File src, File target) throws IOException {
|
|
|
- final long begin = eventHooks.beforeMetadataOp(volume, MOVE);
|
|
|
+ final long begin = profilingEventHook.beforeMetadataOp(volume, MOVE);
|
|
|
try {
|
|
|
+ faultInjectorEventHook.beforeMetadataOp(volume, MOVE);
|
|
|
FileUtil.replaceFile(src, target);
|
|
|
- eventHooks.afterMetadataOp(volume, MOVE, begin);
|
|
|
+ profilingEventHook.afterMetadataOp(volume, MOVE, begin);
|
|
|
} catch(Exception e) {
|
|
|
- eventHooks.onFailure(datanode, volume, MOVE, e, begin);
|
|
|
+ onFailure(volume, begin);
|
|
|
throw e;
|
|
|
}
|
|
|
}
|
|
@@ -530,12 +533,13 @@ public class FileIoProvider {
|
|
|
public void rename(
|
|
|
@Nullable FsVolumeSpi volume, File src, File target)
|
|
|
throws IOException {
|
|
|
- final long begin = eventHooks.beforeMetadataOp(volume, MOVE);
|
|
|
+ final long begin = profilingEventHook.beforeMetadataOp(volume, MOVE);
|
|
|
try {
|
|
|
+ faultInjectorEventHook.beforeMetadataOp(volume, MOVE);
|
|
|
Storage.rename(src, target);
|
|
|
- eventHooks.afterMetadataOp(volume, MOVE, begin);
|
|
|
+ profilingEventHook.afterMetadataOp(volume, MOVE, begin);
|
|
|
} catch(Exception e) {
|
|
|
- eventHooks.onFailure(datanode, volume, MOVE, e, begin);
|
|
|
+ onFailure(volume, begin);
|
|
|
throw e;
|
|
|
}
|
|
|
}
|
|
@@ -552,12 +556,13 @@ public class FileIoProvider {
|
|
|
public void moveFile(
|
|
|
@Nullable FsVolumeSpi volume, File src, File target)
|
|
|
throws IOException {
|
|
|
- final long begin = eventHooks.beforeMetadataOp(volume, MOVE);
|
|
|
+ final long begin = profilingEventHook.beforeMetadataOp(volume, MOVE);
|
|
|
try {
|
|
|
+ faultInjectorEventHook.beforeMetadataOp(volume, MOVE);
|
|
|
FileUtils.moveFile(src, target);
|
|
|
- eventHooks.afterMetadataOp(volume, MOVE, begin);
|
|
|
+ profilingEventHook.afterMetadataOp(volume, MOVE, begin);
|
|
|
} catch(Exception e) {
|
|
|
- eventHooks.onFailure(datanode, volume, MOVE, e, begin);
|
|
|
+ onFailure(volume, begin);
|
|
|
throw e;
|
|
|
}
|
|
|
}
|
|
@@ -576,12 +581,13 @@ public class FileIoProvider {
|
|
|
public void move(
|
|
|
@Nullable FsVolumeSpi volume, Path src, Path target,
|
|
|
CopyOption... options) throws IOException {
|
|
|
- final long begin = eventHooks.beforeMetadataOp(volume, MOVE);
|
|
|
+ final long begin = profilingEventHook.beforeMetadataOp(volume, MOVE);
|
|
|
try {
|
|
|
+ faultInjectorEventHook.beforeMetadataOp(volume, MOVE);
|
|
|
Files.move(src, target, options);
|
|
|
- eventHooks.afterMetadataOp(volume, MOVE, begin);
|
|
|
+ profilingEventHook.afterMetadataOp(volume, MOVE, begin);
|
|
|
} catch(Exception e) {
|
|
|
- eventHooks.onFailure(datanode, volume, MOVE, e, begin);
|
|
|
+ onFailure(volume, begin);
|
|
|
throw e;
|
|
|
}
|
|
|
}
|
|
@@ -600,12 +606,14 @@ public class FileIoProvider {
|
|
|
@Nullable FsVolumeSpi volume, File src, File target,
|
|
|
boolean preserveFileDate) throws IOException {
|
|
|
final long length = src.length();
|
|
|
- final long begin = eventHooks.beforeFileIo(volume, NATIVE_COPY, length);
|
|
|
+ final long begin = profilingEventHook.beforeFileIo(volume, NATIVE_COPY,
|
|
|
+ length);
|
|
|
try {
|
|
|
+ faultInjectorEventHook.beforeFileIo(volume, NATIVE_COPY, length);
|
|
|
Storage.nativeCopyFileUnbuffered(src, target, preserveFileDate);
|
|
|
- eventHooks.afterFileIo(volume, NATIVE_COPY, begin, length);
|
|
|
+ profilingEventHook.afterFileIo(volume, NATIVE_COPY, begin, length);
|
|
|
} catch(Exception e) {
|
|
|
- eventHooks.onFailure(datanode, volume, NATIVE_COPY, e, begin);
|
|
|
+ onFailure(volume, begin);
|
|
|
throw e;
|
|
|
}
|
|
|
}
|
|
@@ -622,15 +630,16 @@ public class FileIoProvider {
|
|
|
*/
|
|
|
public boolean mkdirs(
|
|
|
@Nullable FsVolumeSpi volume, File dir) throws IOException {
|
|
|
- final long begin = eventHooks.beforeMetadataOp(volume, MKDIRS);
|
|
|
+ final long begin = profilingEventHook.beforeMetadataOp(volume, MKDIRS);
|
|
|
boolean created = false;
|
|
|
boolean isDirectory;
|
|
|
try {
|
|
|
+ faultInjectorEventHook.beforeMetadataOp(volume, MKDIRS);
|
|
|
created = dir.mkdirs();
|
|
|
isDirectory = !created && dir.isDirectory();
|
|
|
- eventHooks.afterMetadataOp(volume, MKDIRS, begin);
|
|
|
+ profilingEventHook.afterMetadataOp(volume, MKDIRS, begin);
|
|
|
} catch(Exception e) {
|
|
|
- eventHooks.onFailure(datanode, volume, MKDIRS, e, begin);
|
|
|
+ onFailure(volume, begin);
|
|
|
throw e;
|
|
|
}
|
|
|
|
|
@@ -650,13 +659,14 @@ public class FileIoProvider {
|
|
|
*/
|
|
|
public void mkdirsWithExistsCheck(
|
|
|
@Nullable FsVolumeSpi volume, File dir) throws IOException {
|
|
|
- final long begin = eventHooks.beforeMetadataOp(volume, MKDIRS);
|
|
|
+ final long begin = profilingEventHook.beforeMetadataOp(volume, MKDIRS);
|
|
|
boolean succeeded = false;
|
|
|
try {
|
|
|
+ faultInjectorEventHook.beforeMetadataOp(volume, MKDIRS);
|
|
|
succeeded = dir.isDirectory() || dir.mkdirs();
|
|
|
- eventHooks.afterMetadataOp(volume, MKDIRS, begin);
|
|
|
+ profilingEventHook.afterMetadataOp(volume, MKDIRS, begin);
|
|
|
} catch(Exception e) {
|
|
|
- eventHooks.onFailure(datanode, volume, MKDIRS, e, begin);
|
|
|
+ onFailure(volume, begin);
|
|
|
throw e;
|
|
|
}
|
|
|
|
|
@@ -676,13 +686,14 @@ public class FileIoProvider {
|
|
|
*/
|
|
|
public File[] listFiles(
|
|
|
@Nullable FsVolumeSpi volume, File dir) throws IOException {
|
|
|
- final long begin = eventHooks.beforeMetadataOp(volume, LIST);
|
|
|
+ final long begin = profilingEventHook.beforeMetadataOp(volume, LIST);
|
|
|
try {
|
|
|
+ faultInjectorEventHook.beforeMetadataOp(volume, LIST);
|
|
|
File[] children = FileUtil.listFiles(dir);
|
|
|
- eventHooks.afterMetadataOp(volume, LIST, begin);
|
|
|
+ profilingEventHook.afterMetadataOp(volume, LIST, begin);
|
|
|
return children;
|
|
|
} catch(Exception e) {
|
|
|
- eventHooks.onFailure(datanode, volume, LIST, e, begin);
|
|
|
+ onFailure(volume, begin);
|
|
|
throw e;
|
|
|
}
|
|
|
}
|
|
@@ -698,13 +709,14 @@ public class FileIoProvider {
|
|
|
*/
|
|
|
public String[] list(
|
|
|
@Nullable FsVolumeSpi volume, File dir) throws IOException {
|
|
|
- final long begin = eventHooks.beforeMetadataOp(volume, LIST);
|
|
|
+ final long begin = profilingEventHook.beforeMetadataOp(volume, LIST);
|
|
|
try {
|
|
|
+ faultInjectorEventHook.beforeMetadataOp(volume, LIST);
|
|
|
String[] children = FileUtil.list(dir);
|
|
|
- eventHooks.afterMetadataOp(volume, LIST, begin);
|
|
|
+ profilingEventHook.afterMetadataOp(volume, LIST, begin);
|
|
|
return children;
|
|
|
} catch(Exception e) {
|
|
|
- eventHooks.onFailure(datanode, volume, LIST, e, begin);
|
|
|
+ onFailure(volume, begin);
|
|
|
throw e;
|
|
|
}
|
|
|
}
|
|
@@ -721,13 +733,14 @@ public class FileIoProvider {
|
|
|
public List<String> listDirectory(
|
|
|
@Nullable FsVolumeSpi volume, File dir,
|
|
|
FilenameFilter filter) throws IOException {
|
|
|
- final long begin = eventHooks.beforeMetadataOp(volume, LIST);
|
|
|
+ final long begin = profilingEventHook.beforeMetadataOp(volume, LIST);
|
|
|
try {
|
|
|
+ faultInjectorEventHook.beforeMetadataOp(volume, LIST);
|
|
|
List<String> children = IOUtils.listDirectory(dir, filter);
|
|
|
- eventHooks.afterMetadataOp(volume, LIST, begin);
|
|
|
+ profilingEventHook.afterMetadataOp(volume, LIST, begin);
|
|
|
return children;
|
|
|
} catch(Exception e) {
|
|
|
- eventHooks.onFailure(datanode, volume, LIST, e, begin);
|
|
|
+ onFailure(volume, begin);
|
|
|
throw e;
|
|
|
}
|
|
|
}
|
|
@@ -743,13 +756,14 @@ public class FileIoProvider {
|
|
|
*/
|
|
|
public int getHardLinkCount(
|
|
|
@Nullable FsVolumeSpi volume, File f) throws IOException {
|
|
|
- final long begin = eventHooks.beforeMetadataOp(volume, LIST);
|
|
|
+ final long begin = profilingEventHook.beforeMetadataOp(volume, LIST);
|
|
|
try {
|
|
|
+ faultInjectorEventHook.beforeMetadataOp(volume, LIST);
|
|
|
int count = HardLink.getLinkCount(f);
|
|
|
- eventHooks.afterMetadataOp(volume, LIST, begin);
|
|
|
+ profilingEventHook.afterMetadataOp(volume, LIST, begin);
|
|
|
return count;
|
|
|
} catch(Exception e) {
|
|
|
- eventHooks.onFailure(datanode, volume, LIST, e, begin);
|
|
|
+ onFailure(volume, begin);
|
|
|
throw e;
|
|
|
}
|
|
|
}
|
|
@@ -762,13 +776,14 @@ public class FileIoProvider {
|
|
|
* @return true if the file exists.
|
|
|
*/
|
|
|
public boolean exists(@Nullable FsVolumeSpi volume, File f) {
|
|
|
- final long begin = eventHooks.beforeMetadataOp(volume, EXISTS);
|
|
|
+ final long begin = profilingEventHook.beforeMetadataOp(volume, EXISTS);
|
|
|
try {
|
|
|
+ faultInjectorEventHook.beforeMetadataOp(volume, EXISTS);
|
|
|
boolean exists = f.exists();
|
|
|
- eventHooks.afterMetadataOp(volume, EXISTS, begin);
|
|
|
+ profilingEventHook.afterMetadataOp(volume, EXISTS, begin);
|
|
|
return exists;
|
|
|
} catch(Exception e) {
|
|
|
- eventHooks.onFailure(datanode, volume, EXISTS, e, begin);
|
|
|
+ onFailure(volume, begin);
|
|
|
throw e;
|
|
|
}
|
|
|
}
|
|
@@ -803,13 +818,14 @@ public class FileIoProvider {
|
|
|
*/
|
|
|
@Override
|
|
|
public int read() throws IOException {
|
|
|
- final long begin = eventHooks.beforeFileIo(volume, READ, 1);
|
|
|
+ final long begin = profilingEventHook.beforeFileIo(volume, READ, LEN_INT);
|
|
|
try {
|
|
|
+ faultInjectorEventHook.beforeFileIo(volume, READ, LEN_INT);
|
|
|
int b = super.read();
|
|
|
- eventHooks.afterFileIo(volume, READ, begin, 1);
|
|
|
+ profilingEventHook.afterFileIo(volume, READ, begin, LEN_INT);
|
|
|
return b;
|
|
|
} catch(Exception e) {
|
|
|
- eventHooks.onFailure(datanode, volume, READ, e, begin);
|
|
|
+ onFailure(volume, begin);
|
|
|
throw e;
|
|
|
}
|
|
|
}
|
|
@@ -819,13 +835,15 @@ public class FileIoProvider {
|
|
|
*/
|
|
|
@Override
|
|
|
public int read(@Nonnull byte[] b) throws IOException {
|
|
|
- final long begin = eventHooks.beforeFileIo(volume, READ, b.length);
|
|
|
+ final long begin = profilingEventHook.beforeFileIo(volume, READ, b
|
|
|
+ .length);
|
|
|
try {
|
|
|
+ faultInjectorEventHook.beforeFileIo(volume, READ, b.length);
|
|
|
int numBytesRead = super.read(b);
|
|
|
- eventHooks.afterFileIo(volume, READ, begin, numBytesRead);
|
|
|
+ profilingEventHook.afterFileIo(volume, READ, begin, numBytesRead);
|
|
|
return numBytesRead;
|
|
|
} catch(Exception e) {
|
|
|
- eventHooks.onFailure(datanode, volume, READ, e, begin);
|
|
|
+ onFailure(volume, begin);
|
|
|
throw e;
|
|
|
}
|
|
|
}
|
|
@@ -835,13 +853,14 @@ public class FileIoProvider {
|
|
|
*/
|
|
|
@Override
|
|
|
public int read(@Nonnull byte[] b, int off, int len) throws IOException {
|
|
|
- final long begin = eventHooks.beforeFileIo(volume, READ, len);
|
|
|
+ final long begin = profilingEventHook.beforeFileIo(volume, READ, len);
|
|
|
try {
|
|
|
+ faultInjectorEventHook.beforeFileIo(volume, READ, len);
|
|
|
int numBytesRead = super.read(b, off, len);
|
|
|
- eventHooks.afterFileIo(volume, READ, begin, numBytesRead);
|
|
|
+ profilingEventHook.afterFileIo(volume, READ, begin, numBytesRead);
|
|
|
return numBytesRead;
|
|
|
} catch(Exception e) {
|
|
|
- eventHooks.onFailure(datanode, volume, READ, e, begin);
|
|
|
+ onFailure(volume, begin);
|
|
|
throw e;
|
|
|
}
|
|
|
}
|
|
@@ -878,12 +897,14 @@ public class FileIoProvider {
|
|
|
*/
|
|
|
@Override
|
|
|
public void write(int b) throws IOException {
|
|
|
- final long begin = eventHooks.beforeFileIo(volume, WRITE, 1);
|
|
|
+ final long begin = profilingEventHook.beforeFileIo(volume, WRITE,
|
|
|
+ LEN_INT);
|
|
|
try {
|
|
|
+ faultInjectorEventHook.beforeFileIo(volume, WRITE, LEN_INT);
|
|
|
super.write(b);
|
|
|
- eventHooks.afterFileIo(volume, WRITE, begin, 1);
|
|
|
+ profilingEventHook.afterFileIo(volume, WRITE, begin, LEN_INT);
|
|
|
} catch(Exception e) {
|
|
|
- eventHooks.onFailure(datanode, volume, WRITE, e, begin);
|
|
|
+ onFailure(volume, begin);
|
|
|
throw e;
|
|
|
}
|
|
|
}
|
|
@@ -893,12 +914,14 @@ public class FileIoProvider {
|
|
|
*/
|
|
|
@Override
|
|
|
public void write(@Nonnull byte[] b) throws IOException {
|
|
|
- final long begin = eventHooks.beforeFileIo(volume, WRITE, b.length);
|
|
|
+ final long begin = profilingEventHook.beforeFileIo(volume, WRITE, b
|
|
|
+ .length);
|
|
|
try {
|
|
|
+ faultInjectorEventHook.beforeFileIo(volume, WRITE, b.length);
|
|
|
super.write(b);
|
|
|
- eventHooks.afterFileIo(volume, WRITE, begin, b.length);
|
|
|
+ profilingEventHook.afterFileIo(volume, WRITE, begin, b.length);
|
|
|
} catch(Exception e) {
|
|
|
- eventHooks.onFailure(datanode, volume, WRITE, e, begin);
|
|
|
+ onFailure(volume, begin);
|
|
|
throw e;
|
|
|
}
|
|
|
}
|
|
@@ -908,12 +931,13 @@ public class FileIoProvider {
|
|
|
*/
|
|
|
@Override
|
|
|
public void write(@Nonnull byte[] b, int off, int len) throws IOException {
|
|
|
- final long begin = eventHooks.beforeFileIo(volume, WRITE, len);
|
|
|
+ final long begin = profilingEventHook.beforeFileIo(volume, WRITE, len);
|
|
|
try {
|
|
|
+ faultInjectorEventHook.beforeFileIo(volume, WRITE, len);
|
|
|
super.write(b, off, len);
|
|
|
- eventHooks.afterFileIo(volume, WRITE, begin, len);
|
|
|
+ profilingEventHook.afterFileIo(volume, WRITE, begin, len);
|
|
|
} catch(Exception e) {
|
|
|
- eventHooks.onFailure(datanode, volume, WRITE, e, begin);
|
|
|
+ onFailure(volume, begin);
|
|
|
throw e;
|
|
|
}
|
|
|
}
|
|
@@ -935,77 +959,93 @@ public class FileIoProvider {
|
|
|
|
|
|
@Override
|
|
|
public int read() throws IOException {
|
|
|
- final long begin = eventHooks.beforeFileIo(volume, READ, 1);
|
|
|
+ final long begin = profilingEventHook.beforeFileIo(volume, READ, LEN_INT);
|
|
|
try {
|
|
|
+ faultInjectorEventHook.beforeFileIo(volume, READ, LEN_INT);
|
|
|
int b = super.read();
|
|
|
- eventHooks.afterFileIo(volume, READ, begin, 1);
|
|
|
+ profilingEventHook.afterFileIo(volume, READ, begin, LEN_INT);
|
|
|
return b;
|
|
|
} catch(Exception e) {
|
|
|
- eventHooks.onFailure(datanode, volume, READ, e, begin);
|
|
|
+ onFailure(volume, begin);
|
|
|
throw e;
|
|
|
}
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
public int read(byte[] b, int off, int len) throws IOException {
|
|
|
- final long begin = eventHooks.beforeFileIo(volume, READ, len);
|
|
|
+ final long begin = profilingEventHook.beforeFileIo(volume, READ, len);
|
|
|
try {
|
|
|
+ faultInjectorEventHook.beforeFileIo(volume, READ, len);
|
|
|
int numBytesRead = super.read(b, off, len);
|
|
|
- eventHooks.afterFileIo(volume, READ, begin, numBytesRead);
|
|
|
+ profilingEventHook.afterFileIo(volume, READ, begin, numBytesRead);
|
|
|
return numBytesRead;
|
|
|
} catch(Exception e) {
|
|
|
- eventHooks.onFailure(datanode, volume, READ, e, begin);
|
|
|
+ onFailure(volume, begin);
|
|
|
throw e;
|
|
|
}
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
public int read(byte[] b) throws IOException {
|
|
|
- final long begin = eventHooks.beforeFileIo(volume, READ, b.length);
|
|
|
+ final long begin = profilingEventHook.beforeFileIo(volume, READ, b
|
|
|
+ .length);
|
|
|
try {
|
|
|
+ faultInjectorEventHook.beforeFileIo(volume, READ, b.length);
|
|
|
int numBytesRead = super.read(b);
|
|
|
- eventHooks.afterFileIo(volume, READ, begin, numBytesRead);
|
|
|
+ profilingEventHook.afterFileIo(volume, READ, begin, numBytesRead);
|
|
|
return numBytesRead;
|
|
|
} catch(Exception e) {
|
|
|
- eventHooks.onFailure(datanode, volume, READ, e, begin);
|
|
|
+ onFailure(volume, begin);
|
|
|
throw e;
|
|
|
}
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
public void write(int b) throws IOException {
|
|
|
- final long begin = eventHooks.beforeFileIo(volume, WRITE, 1);
|
|
|
+ final long begin = profilingEventHook.beforeFileIo(volume, WRITE,
|
|
|
+ LEN_INT);
|
|
|
try {
|
|
|
+ faultInjectorEventHook.beforeFileIo(volume, WRITE, LEN_INT);
|
|
|
super.write(b);
|
|
|
- eventHooks.afterFileIo(volume, WRITE, begin, 1);
|
|
|
+ profilingEventHook.afterFileIo(volume, WRITE, begin, LEN_INT);
|
|
|
} catch(Exception e) {
|
|
|
- eventHooks.onFailure(datanode, volume, WRITE, e, begin);
|
|
|
+ onFailure(volume, begin);
|
|
|
throw e;
|
|
|
}
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
public void write(@Nonnull byte[] b) throws IOException {
|
|
|
- final long begin = eventHooks.beforeFileIo(volume, WRITE, b.length);
|
|
|
+ final long begin = profilingEventHook.beforeFileIo(volume, WRITE, b
|
|
|
+ .length);
|
|
|
try {
|
|
|
+ faultInjectorEventHook.beforeFileIo(volume, WRITE, b.length);
|
|
|
super.write(b);
|
|
|
- eventHooks.afterFileIo(volume, WRITE, begin, b.length);
|
|
|
+ profilingEventHook.afterFileIo(volume, WRITE, begin, b.length);
|
|
|
} catch(Exception e) {
|
|
|
- eventHooks.onFailure(datanode, volume, WRITE, e, begin);
|
|
|
+ onFailure(volume, begin);
|
|
|
throw e;
|
|
|
}
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
public void write(byte[] b, int off, int len) throws IOException {
|
|
|
- final long begin = eventHooks.beforeFileIo(volume, WRITE, len);
|
|
|
+ final long begin = profilingEventHook.beforeFileIo(volume, WRITE, len);
|
|
|
try {
|
|
|
+ faultInjectorEventHook.beforeFileIo(volume, WRITE, len);
|
|
|
super.write(b, off, len);
|
|
|
- eventHooks.afterFileIo(volume, WRITE, begin, len);
|
|
|
+ profilingEventHook.afterFileIo(volume, WRITE, begin, len);
|
|
|
} catch(Exception e) {
|
|
|
- eventHooks.onFailure(datanode, volume, WRITE, e, begin);
|
|
|
+ onFailure(volume, begin);
|
|
|
throw e;
|
|
|
}
|
|
|
}
|
|
|
}
|
|
|
+
|
|
|
+ private void onFailure(@Nullable FsVolumeSpi volume, long begin) {
|
|
|
+ if (datanode != null && volume != null) {
|
|
|
+ datanode.checkDiskErrorAsync(volume);
|
|
|
+ }
|
|
|
+ profilingEventHook.onFailure(volume, begin);
|
|
|
+ }
|
|
|
}
|