|
@@ -22,22 +22,20 @@ import java.io.FileDescriptor;
|
|
|
import java.io.FileInputStream;
|
|
|
import java.io.FileOutputStream;
|
|
|
import java.io.IOException;
|
|
|
-import java.io.InputStream;
|
|
|
-import java.io.OutputStream;
|
|
|
import java.io.RandomAccessFile;
|
|
|
import java.lang.reflect.Field;
|
|
|
import java.nio.ByteBuffer;
|
|
|
import java.nio.MappedByteBuffer;
|
|
|
+import java.nio.channels.FileChannel;
|
|
|
import java.util.Map;
|
|
|
import java.util.concurrent.ConcurrentHashMap;
|
|
|
|
|
|
-import org.apache.commons.io.FileUtils;
|
|
|
import org.apache.hadoop.classification.InterfaceAudience;
|
|
|
import org.apache.hadoop.classification.InterfaceStability;
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.fs.CommonConfigurationKeys;
|
|
|
import org.apache.hadoop.fs.HardLink;
|
|
|
-import org.apache.hadoop.fs.Path;
|
|
|
+import org.apache.hadoop.io.IOUtils;
|
|
|
import org.apache.hadoop.io.SecureIOUtils.AlreadyExistsException;
|
|
|
import org.apache.hadoop.util.NativeCodeLoader;
|
|
|
import org.apache.hadoop.util.Shell;
|
|
@@ -662,7 +660,7 @@ public class NativeIO {
|
|
|
* user account name, of the format DOMAIN\UserName. This method
|
|
|
* will remove the domain part of the full logon name.
|
|
|
*
|
|
|
- * @param the full principal name containing the domain
|
|
|
+ * @param Fthe full principal name containing the domain
|
|
|
* @return name with domain removed
|
|
|
*/
|
|
|
private static String stripDomain(String name) {
|
|
@@ -855,24 +853,66 @@ public class NativeIO {
|
|
|
|
|
|
/**
|
|
|
* Unbuffered file copy from src to dst without tainting OS buffer cache
|
|
|
- * In Linux, it uses sendfile() which uses O_DIRECT flag internally
|
|
|
- * In Windows, it uses CopyFileEx with COPY_FILE_NO_BUFFERING flag
|
|
|
*
|
|
|
- * Note: This does not support FreeBSD/OSX which have a different sendfile()
|
|
|
- * semantic. Also, this simple native wrapper does minimal parameter checking
|
|
|
- * It is recommended to use wrapper function like
|
|
|
- * the Storage#nativeCopyFileUnbuffered() function in hadoop-hdfs.
|
|
|
+ * In POSIX platform:
|
|
|
+ * It uses FileChannel#transferTo() which internally attempts
|
|
|
+ * unbuffered IO on OS with native sendfile64() support and falls back to
|
|
|
+ * buffered IO otherwise.
|
|
|
+ *
|
|
|
+ * It minimizes the number of FileChannel#transferTo call by passing the the
|
|
|
+ * src file size directly instead of a smaller size as the 3rd parameter.
|
|
|
+ * This saves the number of sendfile64() system call when native sendfile64()
|
|
|
+ * is supported. In the two fall back cases where sendfile is not supported,
|
|
|
+ * FileChannle#transferTo already has its own batching of size 8 MB and 8 KB,
|
|
|
+ * respectively.
|
|
|
+ *
|
|
|
+ * In Windows Platform:
|
|
|
+ * It uses its own native wrapper of CopyFileEx with COPY_FILE_NO_BUFFERING
|
|
|
+ * flag, which is supported on Windows Server 2008 and above.
|
|
|
+ *
|
|
|
+ * Ideally, we should use FileChannel#transferTo() across both POSIX and Windows
|
|
|
+ * platform. Unfortunately, the wrapper(Java_sun_nio_ch_FileChannelImpl_transferTo0)
|
|
|
+ * used by FileChannel#transferTo for unbuffered IO is not implemented on Windows.
|
|
|
+ * Based on OpenJDK 6/7/8 source code, Java_sun_nio_ch_FileChannelImpl_transferTo0
|
|
|
+ * on Windows simply returns IOS_UNSUPPORTED.
|
|
|
*
|
|
|
+ * Note: This simple native wrapper does minimal parameter checking before copy and
|
|
|
+ * consistency check (e.g., size) after copy.
|
|
|
+ * It is recommended to use wrapper function like
|
|
|
+ * the Storage#nativeCopyFileUnbuffered() function in hadoop-hdfs with pre/post copy
|
|
|
+ * checks.
|
|
|
*
|
|
|
* @param src The source path
|
|
|
* @param dst The destination path
|
|
|
* @throws IOException
|
|
|
*/
|
|
|
public static void copyFileUnbuffered(File src, File dst) throws IOException {
|
|
|
- if ((nativeLoaded) && (Shell.WINDOWS || Shell.LINUX)) {
|
|
|
+ if (nativeLoaded && Shell.WINDOWS) {
|
|
|
copyFileUnbuffered0(src.getAbsolutePath(), dst.getAbsolutePath());
|
|
|
} else {
|
|
|
- FileUtils.copyFile(src, dst);
|
|
|
+ FileInputStream fis = null;
|
|
|
+ FileOutputStream fos = null;
|
|
|
+ FileChannel input = null;
|
|
|
+ FileChannel output = null;
|
|
|
+ try {
|
|
|
+ fis = new FileInputStream(src);
|
|
|
+ fos = new FileOutputStream(dst);
|
|
|
+ input = fis.getChannel();
|
|
|
+ output = fos.getChannel();
|
|
|
+ long remaining = input.size();
|
|
|
+ long position = 0;
|
|
|
+ long transferred = 0;
|
|
|
+ while (remaining > 0) {
|
|
|
+ transferred = input.transferTo(position, remaining, output);
|
|
|
+ remaining -= transferred;
|
|
|
+ position += transferred;
|
|
|
+ }
|
|
|
+ } finally {
|
|
|
+ IOUtils.cleanup(LOG, output);
|
|
|
+ IOUtils.cleanup(LOG, fos);
|
|
|
+ IOUtils.cleanup(LOG, input);
|
|
|
+ IOUtils.cleanup(LOG, fis);
|
|
|
+ }
|
|
|
}
|
|
|
}
|
|
|
|