listLocalStatusIterator(
+ final Path path) throws IOException {
+ return local.listLocatedStatus(path);
}
- final String key = pathToKey(dst);
- final ObjectMetadata om = newObjectMetadata(srcfile.length());
- Progressable progress = null;
- PutObjectRequest putObjectRequest = newPutObjectRequest(key, om, srcfile);
- invoker.retry("copyFromLocalFile(" + src + ")", dst.toString(), true,
- () -> executePut(putObjectRequest, progress));
- if (delSrc) {
- local.delete(src, false);
+
+ @Override
+ public File pathToLocalFile(Path path) {
+ return local.pathToFile(path);
+ }
+
+ @Override
+ public boolean deleteLocal(Path path, boolean recursive) throws IOException {
+ return local.delete(path, recursive);
+ }
+
+ @Override
+ public void copyLocalFileFromTo(File file, Path from, Path to) throws IOException {
+ trackDurationAndSpan(
+ OBJECT_PUT_REQUESTS,
+ to,
+ () -> {
+ final String key = pathToKey(to);
+ final ObjectMetadata om = newObjectMetadata(file.length());
+ Progressable progress = null;
+ PutObjectRequest putObjectRequest = newPutObjectRequest(key, om, file);
+ S3AFileSystem.this.invoker.retry(
+ "putObject(" + "" + ")", to.toString(),
+ true,
+ () -> executePut(putObjectRequest, progress));
+
+ return null;
+ });
+ }
+
+ @Override
+ public FileStatus getFileStatus(Path f) throws IOException {
+ return S3AFileSystem.this.getFileStatus(f);
+ }
+
+ @Override
+ public boolean createEmptyDir(Path path, StoreContext storeContext)
+ throws IOException {
+ return trackDuration(getDurationTrackerFactory(),
+ INVOCATION_MKDIRS.getSymbol(),
+ new MkdirOperation(
+ storeContext,
+ path,
+ createMkdirOperationCallbacks()));
}
}
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/CopyFromLocalOperation.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/CopyFromLocalOperation.java
new file mode 100644
index 0000000000..0a665cd33f
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/CopyFromLocalOperation.java
@@ -0,0 +1,540 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3a.impl;
+
+import java.io.Closeable;
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.Serializable;
+import java.net.URI;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashSet;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.Optional;
+import java.util.Set;
+import java.util.Stack;
+import java.util.concurrent.CompletableFuture;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.commons.collections.comparators.ReverseComparator;
+import org.apache.hadoop.fs.FileAlreadyExistsException;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.LocatedFileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.PathExistsException;
+import org.apache.hadoop.fs.PathIOException;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.fs.s3a.Retries;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.ListeningExecutorService;
+import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.MoreExecutors;
+
+import static org.apache.hadoop.fs.s3a.impl.CallableSupplier.submit;
+import static org.apache.hadoop.fs.s3a.impl.CallableSupplier.waitForCompletion;
+import static org.apache.hadoop.fs.store.audit.AuditingFunctions.callableWithinAuditSpan;
+
+/**
+ * Implementation of CopyFromLocalOperation.
+ *
+ * This operation copies a file or directory (recursively) from a local
+ * FS to an object store. Initially, this operation has been developed for
+ * S3 (s3a) interaction, however, there's minimal work needed for it to
+ * work with other stores.
+ *
+ * How the uploading of files works:
+ *
+ * - all source files and directories are scanned through;
+ * - the LARGEST_N_FILES start uploading;
+ * - the remaining files are shuffled and uploaded;
+ * -
+ * any remaining empty directory is uploaded too to preserve local
+ * tree structure.
+ *
+ *
+ */
+public class CopyFromLocalOperation extends ExecutingStoreOperation {
+
+ /**
+ * Largest N files to be uploaded first.
+ */
+ private static final int LARGEST_N_FILES = 5;
+
+ private static final Logger LOG = LoggerFactory.getLogger(
+ CopyFromLocalOperation.class);
+
+ /**
+ * Callbacks to be used by this operation for external / IO actions.
+ */
+ private final CopyFromLocalOperationCallbacks callbacks;
+
+ /**
+ * Delete source after operation finishes.
+ */
+ private final boolean deleteSource;
+
+ /**
+ * Overwrite destination files / folders.
+ */
+ private final boolean overwrite;
+
+ /**
+ * Source path to file / directory.
+ */
+ private final Path source;
+
+ /**
+ * Async operations executor.
+ */
+ private final ListeningExecutorService executor;
+
+ /**
+ * Destination path.
+ */
+ private Path destination;
+
+ /**
+ * Destination file status.
+ */
+ private FileStatus destStatus;
+
+ public CopyFromLocalOperation(
+ final StoreContext storeContext,
+ Path source,
+ Path destination,
+ boolean deleteSource,
+ boolean overwrite,
+ CopyFromLocalOperationCallbacks callbacks) {
+ super(storeContext);
+ this.callbacks = callbacks;
+ this.deleteSource = deleteSource;
+ this.overwrite = overwrite;
+ this.source = source;
+ this.destination = destination;
+
+ // Capacity of 1 is a safe default for now since transfer manager can also
+ // spawn threads when uploading bigger files.
+ this.executor = MoreExecutors.listeningDecorator(
+ storeContext.createThrottledExecutor(1)
+ );
+ }
+
+ /**
+ * Executes the {@link CopyFromLocalOperation}.
+ *
+ * @throws IOException - if there are any failures with upload or deletion
+ * of files. Check {@link CopyFromLocalOperationCallbacks} for specifics.
+ * @throws PathExistsException - if the path exists and no overwrite flag
+ * is set OR if the source is file and destination is a directory
+ */
+ @Override
+ @Retries.RetryTranslated
+ public Void execute()
+ throws IOException, PathExistsException {
+ LOG.debug("Copying local file from {} to {}", source, destination);
+ File sourceFile = callbacks.pathToLocalFile(source);
+ updateDestStatus(destination);
+
+ // Handles bar/ -> foo/ => foo/bar and bar/ -> foo/bar/ => foo/bar/bar
+ if (getDestStatus().isPresent() && getDestStatus().get().isDirectory()
+ && sourceFile.isDirectory()) {
+ destination = new Path(destination, sourceFile.getName());
+ LOG.debug("Destination updated to: {}", destination);
+ updateDestStatus(destination);
+ }
+
+ checkSource(sourceFile);
+ checkDestination(destination, sourceFile, overwrite);
+ uploadSourceFromFS();
+
+ if (deleteSource) {
+ callbacks.deleteLocal(source, true);
+ }
+
+ return null;
+ }
+
+ /**
+ * Does a {@link CopyFromLocalOperationCallbacks#getFileStatus(Path)}
+ * operation on the provided destination and updates the internal status of
+ * destStatus field.
+ *
+ * @param dest - destination Path
+ * @throws IOException if getFileStatus fails
+ */
+ private void updateDestStatus(Path dest) throws IOException {
+ try {
+ destStatus = callbacks.getFileStatus(dest);
+ } catch (FileNotFoundException e) {
+ destStatus = null;
+ }
+ }
+
+ /**
+ * Starts async upload operations for files. Creating an empty directory
+ * classifies as a "file upload".
+ *
+ * Check {@link CopyFromLocalOperation} for details on the order of
+ * operations.
+ *
+ * @throws IOException - if listing or upload fail
+ */
+ private void uploadSourceFromFS() throws IOException {
+ RemoteIterator localFiles = listFilesAndDirs(source);
+ List> activeOps = new ArrayList<>();
+
+ // After all files are traversed, this set will contain only emptyDirs
+ Set emptyDirs = new HashSet<>();
+ List entries = new ArrayList<>();
+ while (localFiles.hasNext()) {
+ LocatedFileStatus sourceFile = localFiles.next();
+ Path sourceFilePath = sourceFile.getPath();
+
+ // Directory containing this file / directory isn't empty
+ emptyDirs.remove(sourceFilePath.getParent());
+
+ if (sourceFile.isDirectory()) {
+ emptyDirs.add(sourceFilePath);
+ continue;
+ }
+
+ Path destPath = getFinalPath(sourceFilePath);
+ // UploadEntries: have a destination path, a file size
+ entries.add(new UploadEntry(
+ sourceFilePath,
+ destPath,
+ sourceFile.getLen()));
+ }
+
+ if (localFiles instanceof Closeable) {
+ IOUtils.closeStream((Closeable) localFiles);
+ }
+
+ // Sort all upload entries based on size
+ entries.sort(new ReverseComparator(new UploadEntry.SizeComparator()));
+
+ // Take only top most N entries and upload
+ final int sortedUploadsCount = Math.min(LARGEST_N_FILES, entries.size());
+ List markedForUpload = new ArrayList<>();
+
+ for (int uploadNo = 0; uploadNo < sortedUploadsCount; uploadNo++) {
+ UploadEntry uploadEntry = entries.get(uploadNo);
+ File file = callbacks.pathToLocalFile(uploadEntry.source);
+ activeOps.add(submitUpload(file, uploadEntry));
+ markedForUpload.add(uploadEntry);
+ }
+
+ // No files found, it's empty source directory
+ if (entries.isEmpty()) {
+ emptyDirs.add(source);
+ }
+
+ // Shuffle all remaining entries and upload them
+ entries.removeAll(markedForUpload);
+ Collections.shuffle(entries);
+ for (UploadEntry uploadEntry : entries) {
+ File file = callbacks.pathToLocalFile(uploadEntry.source);
+ activeOps.add(submitUpload(file, uploadEntry));
+ }
+
+ for (Path emptyDir : emptyDirs) {
+ Path emptyDirPath = getFinalPath(emptyDir);
+ activeOps.add(submitCreateEmptyDir(emptyDirPath));
+ }
+
+ waitForCompletion(activeOps);
+ }
+
+ /**
+ * Async call to create an empty directory.
+ *
+ * @param dir directory path
+ * @return the submitted future
+ */
+ private CompletableFuture submitCreateEmptyDir(Path dir) {
+ return submit(executor, callableWithinAuditSpan(
+ getAuditSpan(), () -> {
+ callbacks.createEmptyDir(dir, getStoreContext());
+ return null;
+ }
+ ));
+ }
+
+ /**
+ * Async call to upload a file.
+ *
+ * @param file - File to be uploaded
+ * @param uploadEntry - Upload entry holding the source and destination
+ * @return the submitted future
+ */
+ private CompletableFuture submitUpload(
+ File file,
+ UploadEntry uploadEntry) {
+ return submit(executor, callableWithinAuditSpan(
+ getAuditSpan(), () -> {
+ callbacks.copyLocalFileFromTo(
+ file,
+ uploadEntry.source,
+ uploadEntry.destination);
+ return null;
+ }
+ ));
+ }
+
+ /**
+ * Checks the source before upload starts.
+ *
+ * @param src - Source file
+ * @throws FileNotFoundException - if the file isn't found
+ */
+ private void checkSource(File src)
+ throws FileNotFoundException {
+ if (!src.exists()) {
+ throw new FileNotFoundException("No file: " + src.getPath());
+ }
+ }
+
+ /**
+ * Check the destination path and make sure it's compatible with the source,
+ * i.e. source and destination are both files / directories.
+ *
+ * @param dest - Destination path
+ * @param src - Source file
+ * @param overwrite - Should source overwrite destination
+ * @throws PathExistsException - If the destination path exists and no
+ * overwrite flag is set
+ * @throws FileAlreadyExistsException - If source is file and destination is path
+ */
+ private void checkDestination(
+ Path dest,
+ File src,
+ boolean overwrite) throws PathExistsException,
+ FileAlreadyExistsException {
+ if (!getDestStatus().isPresent()) {
+ return;
+ }
+
+ if (src.isDirectory() && getDestStatus().get().isFile()) {
+ throw new FileAlreadyExistsException(
+ "Source '" + src.getPath() + "' is directory and " +
+ "destination '" + dest + "' is file");
+ }
+
+ if (!overwrite) {
+ throw new PathExistsException(dest + " already exists");
+ }
+ }
+
+ /**
+ * Get the final path of a source file with regards to its destination.
+ *
+ * @param src - source path
+ * @return - the final path for the source file to be uploaded to
+ * @throws PathIOException - if a relative path can't be created
+ */
+ private Path getFinalPath(Path src) throws PathIOException {
+ URI currentSrcUri = src.toUri();
+ URI relativeSrcUri = source.toUri().relativize(currentSrcUri);
+ if (relativeSrcUri.equals(currentSrcUri)) {
+ throw new PathIOException("Cannot get relative path for URI:"
+ + relativeSrcUri);
+ }
+
+ Optional status = getDestStatus();
+ if (!relativeSrcUri.getPath().isEmpty()) {
+ return new Path(destination, relativeSrcUri.getPath());
+ } else if (status.isPresent() && status.get().isDirectory()) {
+ // file to dir
+ return new Path(destination, src.getName());
+ } else {
+ // file to file
+ return destination;
+ }
+ }
+
+ private Optional getDestStatus() {
+ return Optional.ofNullable(destStatus);
+ }
+
+ /**
+ * {@link RemoteIterator} which lists all of the files and directories for
+ * a given path. It's strikingly similar to
+ * {@link org.apache.hadoop.fs.LocalFileSystem#listFiles(Path, boolean)}
+ * however with the small addition that it includes directories.
+ *
+ * @param path - Path to list files and directories from
+ * @return - an iterator
+ * @throws IOException - if listing of a path file fails
+ */
+ private RemoteIterator listFilesAndDirs(Path path)
+ throws IOException {
+ return new RemoteIterator() {
+ private final Stack> iterators =
+ new Stack<>();
+ private RemoteIterator current =
+ callbacks.listLocalStatusIterator(path);
+ private LocatedFileStatus curFile;
+
+ @Override
+ public boolean hasNext() throws IOException {
+ while (curFile == null) {
+ if (current.hasNext()) {
+ handleFileStat(current.next());
+ } else if (!iterators.empty()) {
+ current = iterators.pop();
+ } else {
+ return false;
+ }
+ }
+ return true;
+ }
+
+ /**
+ * Process the input stat.
+ * If it is a file or directory return the file stat.
+ * If it is a directory, traverse the directory;
+ * @param stat input status
+ * @throws IOException if any IO error occurs
+ */
+ private void handleFileStat(LocatedFileStatus stat)
+ throws IOException {
+ if (stat.isFile()) { // file
+ curFile = stat;
+ } else { // directory
+ curFile = stat;
+ iterators.push(current);
+ current = callbacks.listLocalStatusIterator(stat.getPath());
+ }
+ }
+
+ @Override
+ public LocatedFileStatus next() throws IOException {
+ if (hasNext()) {
+ LocatedFileStatus result = curFile;
+ curFile = null;
+ return result;
+ }
+ throw new NoSuchElementException("No more entry in "
+ + path);
+ }
+ };
+ }
+
+ /**
+ * Represents an entry for a file to be moved.
+ *
+ * Helpful with sorting files by their size and keeping track of path
+ * information for the upload.
+ *
+ */
+ private static final class UploadEntry {
+ private final Path source;
+ private final Path destination;
+ private final long size;
+
+ private UploadEntry(Path source, Path destination, long size) {
+ this.source = source;
+ this.destination = destination;
+ this.size = size;
+ }
+
+ /**
+ * Compares {@link UploadEntry} objects and produces DESC ordering.
+ */
+ static class SizeComparator implements Comparator,
+ Serializable {
+ @Override
+ public int compare(UploadEntry entry1, UploadEntry entry2) {
+ return Long.compare(entry1.size, entry2.size);
+ }
+ }
+ }
+
+ /**
+ * Define the contract for {@link CopyFromLocalOperation} to interact
+ * with any external resources.
+ */
+ public interface CopyFromLocalOperationCallbacks {
+ /**
+ * List all entries (files AND directories) for a path.
+ *
+ * @param path - path to list
+ * @return an iterator for all entries
+ * @throws IOException - for any failure
+ */
+ RemoteIterator listLocalStatusIterator(Path path)
+ throws IOException;
+
+ /**
+ * Get the file status for a path.
+ *
+ * @param path - target path
+ * @return FileStatus
+ * @throws IOException - for any failure
+ */
+ FileStatus getFileStatus(Path path) throws IOException;
+
+ /**
+ * Get the file from a path.
+ *
+ * @param path - target path
+ * @return file at path
+ */
+ File pathToLocalFile(Path path);
+
+ /**
+ * Delete file / directory at path.
+ *
+ * @param path - target path
+ * @param recursive - recursive deletion
+ * @return boolean result of operation
+ * @throws IOException for any failure
+ */
+ boolean deleteLocal(Path path, boolean recursive) throws IOException;
+
+ /**
+ * Copy / Upload a file from a source path to a destination path.
+ *
+ * @param file - target file
+ * @param source - source path
+ * @param destination - destination path
+ * @throws IOException for any failure
+ */
+ void copyLocalFileFromTo(
+ File file,
+ Path source,
+ Path destination) throws IOException;
+
+ /**
+ * Create empty directory at path. Most likely an upload operation.
+ *
+ * @param path - target path
+ * @param storeContext - store context
+ * @return boolean result of operation
+ * @throws IOException for any failure
+ */
+ boolean createEmptyDir(Path path, StoreContext storeContext)
+ throws IOException;
+ }
+}
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ACopyFromLocalFile.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ACopyFromLocalFile.java
index 668e129d57..dfac771dd7 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ACopyFromLocalFile.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ACopyFromLocalFile.java
@@ -19,143 +19,41 @@
package org.apache.hadoop.fs.s3a;
import java.io.File;
-import java.io.FileNotFoundException;
-import java.io.IOException;
-import java.nio.charset.Charset;
-import java.nio.charset.StandardCharsets;
-import org.junit.Ignore;
-import org.junit.Test;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractCopyFromLocalTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+import org.apache.hadoop.fs.contract.s3a.S3AContract;
-import org.apache.commons.io.FileUtils;
-import org.apache.commons.io.IOUtils;
-import org.apache.hadoop.fs.FileAlreadyExistsException;
-import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.PathExistsException;
+import org.junit.Test;
import static org.apache.hadoop.test.LambdaTestUtils.intercept;
-/**
- * Test {@link S3AFileSystem#copyFromLocalFile(boolean, boolean, Path, Path)}.
- * Some of the tests have been disabled pending a fix for HADOOP-15932 and
- * recursive directory copying; the test cases themselves may be obsolete.
- */
-public class ITestS3ACopyFromLocalFile extends AbstractS3ATestBase {
- private static final Charset ASCII = StandardCharsets.US_ASCII;
-
- private File file;
+public class ITestS3ACopyFromLocalFile extends
+ AbstractContractCopyFromLocalTest {
@Override
- public void teardown() throws Exception {
- super.teardown();
- if (file != null) {
- file.delete();
- }
+ protected AbstractFSContract createContract(Configuration conf) {
+ return new S3AContract(conf);
}
- @Test
- public void testCopyEmptyFile() throws Throwable {
- file = File.createTempFile("test", ".txt");
- Path dest = upload(file, true);
- assertPathExists("uploaded file", dest);
- }
-
- @Test
- public void testCopyFile() throws Throwable {
- String message = "hello";
- file = createTempFile(message);
- Path dest = upload(file, true);
- assertPathExists("uploaded file not found", dest);
- S3AFileSystem fs = getFileSystem();
- FileStatus status = fs.getFileStatus(dest);
- assertEquals("File length of " + status,
- message.getBytes(ASCII).length, status.getLen());
- assertFileTextEquals(dest, message);
- }
-
- public void assertFileTextEquals(Path path, String expected)
- throws IOException {
- assertEquals("Wrong data in " + path,
- expected, IOUtils.toString(getFileSystem().open(path), ASCII));
- }
-
- @Test
- public void testCopyFileNoOverwrite() throws Throwable {
- file = createTempFile("hello");
- Path dest = upload(file, true);
- // HADOOP-15932: the exception type changes here
- intercept(PathExistsException.class,
- () -> upload(file, false));
- }
-
- @Test
- public void testCopyFileOverwrite() throws Throwable {
- file = createTempFile("hello");
- Path dest = upload(file, true);
- String updated = "updated";
- FileUtils.write(file, updated, ASCII);
- upload(file, true);
- assertFileTextEquals(dest, updated);
- }
-
- @Test
- @Ignore("HADOOP-15932")
- public void testCopyFileNoOverwriteDirectory() throws Throwable {
- file = createTempFile("hello");
- Path dest = upload(file, true);
- S3AFileSystem fs = getFileSystem();
- fs.delete(dest, false);
- fs.mkdirs(dest);
- intercept(FileAlreadyExistsException.class,
- () -> upload(file, true));
- }
-
- @Test
- public void testCopyMissingFile() throws Throwable {
- file = File.createTempFile("test", ".txt");
- file.delete();
- // first upload to create
- intercept(FileNotFoundException.class, "",
- () -> upload(file, true));
- }
-
- @Test
- @Ignore("HADOOP-15932")
- public void testCopyDirectoryFile() throws Throwable {
- file = File.createTempFile("test", ".txt");
- // first upload to create
- intercept(FileNotFoundException.class, "Not a file",
- () -> upload(file.getParentFile(), true));
- }
-
-
@Test
public void testLocalFilesOnly() throws Throwable {
- Path dst = path("testLocalFilesOnly");
+ describe("Copying into other file systems must fail");
+ Path dest = fileToPath(createTempDirectory("someDir"));
+
intercept(IllegalArgumentException.class,
- () -> {
- getFileSystem().copyFromLocalFile(false, true, dst, dst);
- return "copy successful";
- });
+ () -> getFileSystem().copyFromLocalFile(false, true, dest, dest));
}
- public Path upload(File srcFile, boolean overwrite) throws IOException {
- Path src = new Path(srcFile.toURI());
- Path dst = path(srcFile.getName());
- getFileSystem().copyFromLocalFile(false, overwrite, src, dst);
- return dst;
- }
+ @Test
+ public void testOnlyFromLocal() throws Throwable {
+ describe("Copying must be from a local file system");
+ File source = createTempFile("someFile");
+ Path dest = copyFromLocal(source, true);
- /**
- * Create a temp file with some text.
- * @param text text for the file
- * @return the file
- * @throws IOException on a failure
- */
- public File createTempFile(String text) throws IOException {
- File f = File.createTempFile("test", ".txt");
- FileUtils.write(f, text, ASCII);
- return f;
+ intercept(IllegalArgumentException.class,
+ () -> getFileSystem().copyFromLocalFile(true, true, dest, dest));
}
}