steveloughran commented on a change in pull request #3101:
URL: https://github.com/apache/hadoop/pull/3101#discussion_r658819030
##########
File path:
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ACopyFromLocalFile.java
##########
@@ -120,15 +117,48 @@ public void testCopyMissingFile() throws Throwable {
() -> upload(file, true));
}
+ /*
+ * The following path is being created on disk and copied over
+ * /parent/ (trailing slash to make it clear it's a directory
+ * /parent/test1.txt
+ * /parent/child/test.txt
+ */
@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));
+ public void testCopyTreeDirectoryWithoutDelete() throws Throwable {
+ java.nio.file.Path srcDir = Files.createTempDirectory("parent");
+ java.nio.file.Path childDir = Files.createTempDirectory(srcDir, "child");
+ java.nio.file.Path parentFile = Files.createTempFile(srcDir, "test1",
".txt");
+ java.nio.file.Path childFile = Files.createTempFile(childDir, "test2",
".txt");
+
+ Path src = new Path(srcDir.toUri());
+ Path dst = path(srcDir.getFileName().toString());
+ getFileSystem().copyFromLocalFile(false, true, src, dst);
+
+ java.nio.file.Path parent = srcDir.getParent();
+
+ assertPathExists("Parent directory", srcDir, parent);
+ assertPathExists("Child directory", childDir, parent);
+ assertPathExists("Parent file", parentFile, parent);
+ assertPathExists("Child file", childFile, parent);
+
+ if (!Files.exists(srcDir)) {
Review comment:
just use an assert
##########
File path:
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/CopyFromLocalOperation.java
##########
@@ -0,0 +1,241 @@
+package org.apache.hadoop.fs.s3a.impl;
+
+import org.apache.commons.collections.comparators.ReverseComparator;
+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.RemoteIterator;
+import org.apache.hadoop.fs.s3a.Retries;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+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.Optional;
+import java.util.Set;
+
+/**
+ * TODO list:
+ * - Improve implementation to use Completable Futures
+ * - Better error handling
+ * - Add abstract class + tests for LocalFS
+ * - Add tests for this class
+ * - Add documentation
+ * - This class
+ * - `filesystem.md`
+ * - Clean old `innerCopyFromLocalFile` code up
+ */
+public class CopyFromLocalOperation extends ExecutingStoreOperation<Void> {
+
+ private static final Logger LOG = LoggerFactory.getLogger(
+ CopyFromLocalOperation.class);
+
+ private final CopyFromLocalOperationCallbacks callbacks;
+ private final boolean deleteSource;
+ private final boolean overwrite;
+ private final Path source;
+ private final Path destination;
+
+ private FileStatus dstStatus;
+
+ 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;
+ }
+
+ @Override
+ @Retries.RetryTranslated
+ public Void execute()
+ throws IOException, PathExistsException {
+ LOG.debug("Copying local file from {} to {}", source, destination);
+ File sourceFile = callbacks.pathToFile(source);
+ try {
+ dstStatus = callbacks.getFileStatus(destination);
+ } catch (FileNotFoundException e) {
+ dstStatus = null;
+ }
+
+ checkSource(sourceFile);
+ prepareDestination(destination, sourceFile, overwrite);
+ uploadSourceFromFS();
+
+ if (deleteSource) {
+ callbacks.delete(source, true);
+ }
+
+ return null;
+ }
+
+ private void uploadSourceFromFS()
+ throws IOException, PathExistsException {
+ RemoteIterator<LocatedFileStatus> localFiles = callbacks
+ .listStatusIterator(source, true);
+
+ // After all files are traversed, this set will contain only emptyDirs
+ Set<Path> emptyDirs = new HashSet<>();
+ List<UploadEntry> 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) {
+ ((Closeable) localFiles).close();
+ }
+
+ // Sort all upload entries based on size
+ entries.sort(new ReverseComparator(new UploadEntry.SizeComparator()));
+
+ int LARGEST_N_FILES = 5;
+ final int sortedUploadsCount = Math.min(LARGEST_N_FILES,
entries.size());
+ List<UploadEntry> uploaded = new ArrayList<>();
+
+ // Take only top most X entries and upload
+ for (int uploadNo = 0; uploadNo < sortedUploadsCount; uploadNo++) {
Review comment:
Is this parallelized? I know that the xfer manager does it across
threads, but this is still uploading one by one. This is probably simplest to
start with...
##########
File path:
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/CopyFromLocalOperation.java
##########
@@ -0,0 +1,241 @@
+package org.apache.hadoop.fs.s3a.impl;
+
+import org.apache.commons.collections.comparators.ReverseComparator;
+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.RemoteIterator;
+import org.apache.hadoop.fs.s3a.Retries;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+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.Optional;
+import java.util.Set;
+
+/**
+ * TODO list:
+ * - Improve implementation to use Completable Futures
+ * - Better error handling
+ * - Add abstract class + tests for LocalFS
+ * - Add tests for this class
+ * - Add documentation
+ * - This class
+ * - `filesystem.md`
+ * - Clean old `innerCopyFromLocalFile` code up
+ */
+public class CopyFromLocalOperation extends ExecutingStoreOperation<Void> {
+
+ private static final Logger LOG = LoggerFactory.getLogger(
+ CopyFromLocalOperation.class);
+
+ private final CopyFromLocalOperationCallbacks callbacks;
+ private final boolean deleteSource;
+ private final boolean overwrite;
+ private final Path source;
+ private final Path destination;
+
+ private FileStatus dstStatus;
+
+ 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;
+ }
+
+ @Override
+ @Retries.RetryTranslated
+ public Void execute()
+ throws IOException, PathExistsException {
+ LOG.debug("Copying local file from {} to {}", source, destination);
+ File sourceFile = callbacks.pathToFile(source);
+ try {
+ dstStatus = callbacks.getFileStatus(destination);
+ } catch (FileNotFoundException e) {
+ dstStatus = null;
+ }
+
+ checkSource(sourceFile);
+ prepareDestination(destination, sourceFile, overwrite);
+ uploadSourceFromFS();
+
+ if (deleteSource) {
+ callbacks.delete(source, true);
+ }
+
+ return null;
+ }
+
+ private void uploadSourceFromFS()
+ throws IOException, PathExistsException {
+ RemoteIterator<LocatedFileStatus> localFiles = callbacks
+ .listStatusIterator(source, true);
+
+ // After all files are traversed, this set will contain only emptyDirs
+ Set<Path> emptyDirs = new HashSet<>();
+ List<UploadEntry> 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) {
+ ((Closeable) localFiles).close();
+ }
+
+ // Sort all upload entries based on size
+ entries.sort(new ReverseComparator(new UploadEntry.SizeComparator()));
+
+ int LARGEST_N_FILES = 5;
+ final int sortedUploadsCount = Math.min(LARGEST_N_FILES,
entries.size());
+ List<UploadEntry> uploaded = new ArrayList<>();
+
+ // Take only top most X entries and upload
+ for (int uploadNo = 0; uploadNo < sortedUploadsCount; uploadNo++) {
+ UploadEntry uploadEntry = entries.get(uploadNo);
+ File file = callbacks.pathToFile(uploadEntry.source);
+ callbacks.copyFileFromTo(
+ file,
+ uploadEntry.source,
+ uploadEntry.destination);
+
+ uploaded.add(uploadEntry);
+ }
+
+ // Shuffle all remaining entries and upload them
+ entries.removeAll(uploaded);
+ Collections.shuffle(entries);
+ for (UploadEntry uploadEntry : entries) {
+ File file = callbacks.pathToFile(uploadEntry.source);
+ callbacks.copyFileFromTo(
+ file,
+ uploadEntry.source,
+ uploadEntry.destination);
+ }
+
+ for (Path emptyDir : emptyDirs) {
+ callbacks.createEmptyDir(getFinalPath(emptyDir));
+ }
+ }
+
+ private void checkSource(File src)
+ throws FileNotFoundException {
+ if (!src.exists()) {
+ throw new FileNotFoundException("No file: " + src.getPath());
+ }
+ }
+
+ private void prepareDestination(
+ Path dst,
+ File src,
+ boolean overwrite) throws PathExistsException, IOException {
+ if (!getDstStatus().isPresent()) {
Review comment:
nit, use "dest", we don't pay Oracle extra for vowels
##########
File path:
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ACopyFromLocalFile.java
##########
@@ -120,15 +117,48 @@ public void testCopyMissingFile() throws Throwable {
() -> upload(file, true));
}
+ /*
+ * The following path is being created on disk and copied over
+ * /parent/ (trailing slash to make it clear it's a directory
+ * /parent/test1.txt
+ * /parent/child/test.txt
+ */
@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));
+ public void testCopyTreeDirectoryWithoutDelete() throws Throwable {
+ java.nio.file.Path srcDir = Files.createTempDirectory("parent");
+ java.nio.file.Path childDir = Files.createTempDirectory(srcDir, "child");
+ java.nio.file.Path parentFile = Files.createTempFile(srcDir, "test1",
".txt");
+ java.nio.file.Path childFile = Files.createTempFile(childDir, "test2",
".txt");
+
+ Path src = new Path(srcDir.toUri());
+ Path dst = path(srcDir.getFileName().toString());
+ getFileSystem().copyFromLocalFile(false, true, src, dst);
+
+ java.nio.file.Path parent = srcDir.getParent();
+
+ assertPathExists("Parent directory", srcDir, parent);
+ assertPathExists("Child directory", childDir, parent);
+ assertPathExists("Parent file", parentFile, parent);
+ assertPathExists("Child file", childFile, parent);
+
+ if (!Files.exists(srcDir)) {
+ throw new Exception("Folder was deleted when it shouldn't have!");
+ }
}
+ @Test
+ public void testCopyDirectoryWithDelete() throws Throwable {
+ java.nio.file.Path srcDir = Files.createTempDirectory("parent");
+ Files.createTempFile(srcDir, "test1", ".txt");
+
+ Path src = new Path(srcDir.toUri());
+ Path dst = path(srcDir.getFileName().toString());
+ getFileSystem().copyFromLocalFile(true, true, src, dst);
+
+ if (Files.exists(srcDir)) {
Review comment:
again, assertion
##########
File path:
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/CopyFromLocalOperation.java
##########
@@ -0,0 +1,241 @@
+package org.apache.hadoop.fs.s3a.impl;
+
+import org.apache.commons.collections.comparators.ReverseComparator;
+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.RemoteIterator;
+import org.apache.hadoop.fs.s3a.Retries;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+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.Optional;
+import java.util.Set;
+
+/**
+ * TODO list:
+ * - Improve implementation to use Completable Futures
+ * - Better error handling
+ * - Add abstract class + tests for LocalFS
+ * - Add tests for this class
+ * - Add documentation
+ * - This class
+ * - `filesystem.md`
+ * - Clean old `innerCopyFromLocalFile` code up
+ */
+public class CopyFromLocalOperation extends ExecutingStoreOperation<Void> {
+
+ private static final Logger LOG = LoggerFactory.getLogger(
+ CopyFromLocalOperation.class);
+
+ private final CopyFromLocalOperationCallbacks callbacks;
+ private final boolean deleteSource;
+ private final boolean overwrite;
+ private final Path source;
+ private final Path destination;
+
+ private FileStatus dstStatus;
+
+ 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;
+ }
+
+ @Override
+ @Retries.RetryTranslated
+ public Void execute()
+ throws IOException, PathExistsException {
+ LOG.debug("Copying local file from {} to {}", source, destination);
+ File sourceFile = callbacks.pathToFile(source);
+ try {
+ dstStatus = callbacks.getFileStatus(destination);
+ } catch (FileNotFoundException e) {
+ dstStatus = null;
+ }
+
+ checkSource(sourceFile);
+ prepareDestination(destination, sourceFile, overwrite);
+ uploadSourceFromFS();
+
+ if (deleteSource) {
+ callbacks.delete(source, true);
+ }
+
+ return null;
+ }
+
+ private void uploadSourceFromFS()
+ throws IOException, PathExistsException {
+ RemoteIterator<LocatedFileStatus> localFiles = callbacks
+ .listStatusIterator(source, true);
+
+ // After all files are traversed, this set will contain only emptyDirs
+ Set<Path> emptyDirs = new HashSet<>();
+ List<UploadEntry> 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) {
Review comment:
`IOUtils.closeStream`
##########
File path:
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/CopyFromLocalOperation.java
##########
@@ -0,0 +1,241 @@
+package org.apache.hadoop.fs.s3a.impl;
+
+import org.apache.commons.collections.comparators.ReverseComparator;
+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.RemoteIterator;
+import org.apache.hadoop.fs.s3a.Retries;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+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.Optional;
+import java.util.Set;
+
+/**
+ * TODO list:
+ * - Improve implementation to use Completable Futures
+ * - Better error handling
+ * - Add abstract class + tests for LocalFS
+ * - Add tests for this class
+ * - Add documentation
+ * - This class
+ * - `filesystem.md`
+ * - Clean old `innerCopyFromLocalFile` code up
+ */
+public class CopyFromLocalOperation extends ExecutingStoreOperation<Void> {
+
+ private static final Logger LOG = LoggerFactory.getLogger(
+ CopyFromLocalOperation.class);
+
+ private final CopyFromLocalOperationCallbacks callbacks;
+ private final boolean deleteSource;
+ private final boolean overwrite;
+ private final Path source;
+ private final Path destination;
+
+ private FileStatus dstStatus;
+
+ 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;
+ }
+
+ @Override
+ @Retries.RetryTranslated
+ public Void execute()
+ throws IOException, PathExistsException {
+ LOG.debug("Copying local file from {} to {}", source, destination);
+ File sourceFile = callbacks.pathToFile(source);
+ try {
+ dstStatus = callbacks.getFileStatus(destination);
+ } catch (FileNotFoundException e) {
+ dstStatus = null;
+ }
+
+ checkSource(sourceFile);
+ prepareDestination(destination, sourceFile, overwrite);
+ uploadSourceFromFS();
+
+ if (deleteSource) {
+ callbacks.delete(source, true);
+ }
+
+ return null;
+ }
+
+ private void uploadSourceFromFS()
+ throws IOException, PathExistsException {
+ RemoteIterator<LocatedFileStatus> localFiles = callbacks
+ .listStatusIterator(source, true);
+
+ // After all files are traversed, this set will contain only emptyDirs
+ Set<Path> emptyDirs = new HashSet<>();
+ List<UploadEntry> 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) {
+ ((Closeable) localFiles).close();
+ }
+
+ // Sort all upload entries based on size
+ entries.sort(new ReverseComparator(new UploadEntry.SizeComparator()));
+
+ int LARGEST_N_FILES = 5;
+ final int sortedUploadsCount = Math.min(LARGEST_N_FILES,
entries.size());
+ List<UploadEntry> uploaded = new ArrayList<>();
+
+ // Take only top most X entries and upload
+ for (int uploadNo = 0; uploadNo < sortedUploadsCount; uploadNo++) {
+ UploadEntry uploadEntry = entries.get(uploadNo);
+ File file = callbacks.pathToFile(uploadEntry.source);
+ callbacks.copyFileFromTo(
+ file,
+ uploadEntry.source,
+ uploadEntry.destination);
+
+ uploaded.add(uploadEntry);
+ }
+
+ // Shuffle all remaining entries and upload them
+ entries.removeAll(uploaded);
+ Collections.shuffle(entries);
+ for (UploadEntry uploadEntry : entries) {
+ File file = callbacks.pathToFile(uploadEntry.source);
+ callbacks.copyFileFromTo(
+ file,
+ uploadEntry.source,
+ uploadEntry.destination);
+ }
+
+ for (Path emptyDir : emptyDirs) {
+ callbacks.createEmptyDir(getFinalPath(emptyDir));
+ }
+ }
+
+ private void checkSource(File src)
+ throws FileNotFoundException {
+ if (!src.exists()) {
+ throw new FileNotFoundException("No file: " + src.getPath());
+ }
+ }
+
+ private void prepareDestination(
+ Path dst,
+ File src,
+ boolean overwrite) throws PathExistsException, IOException {
+ if (!getDstStatus().isPresent()) {
+ return;
+ }
+
+ if (src.isFile() && getDstStatus().get().isDirectory()) {
+ throw new PathExistsException(
+ "Source '" + src.getPath() +"' is file and " +
+ "destination '" + dst + "' is directory");
+ }
+
+ if (!overwrite) {
+ throw new PathExistsException(dst + " already exists");
+ }
+ }
+
+ private Path getFinalPath(Path src) throws IOException {
+ URI currentSrcUri = src.toUri();
+ URI relativeSrcUri = source.toUri().relativize(currentSrcUri);
+ if (currentSrcUri == relativeSrcUri) {
+ throw new IOException("Cannot get relative path");
Review comment:
1. use .equals()
1. PathIOException with path
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]