Return-Path: Delivered-To: apmail-lucene-hadoop-commits-archive@locus.apache.org Received: (qmail 2995 invoked from network); 22 Jun 2007 22:48:09 -0000 Received: from hermes.apache.org (HELO mail.apache.org) (140.211.11.2) by minotaur.apache.org with SMTP; 22 Jun 2007 22:48:09 -0000 Received: (qmail 37054 invoked by uid 500); 22 Jun 2007 22:48:13 -0000 Delivered-To: apmail-lucene-hadoop-commits-archive@lucene.apache.org Received: (qmail 37022 invoked by uid 500); 22 Jun 2007 22:48:13 -0000 Mailing-List: contact hadoop-commits-help@lucene.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: hadoop-dev@lucene.apache.org Delivered-To: mailing list hadoop-commits@lucene.apache.org Received: (qmail 37012 invoked by uid 99); 22 Jun 2007 22:48:12 -0000 Received: from herse.apache.org (HELO herse.apache.org) (140.211.11.133) by apache.org (qpsmtpd/0.29) with ESMTP; Fri, 22 Jun 2007 15:48:12 -0700 X-ASF-Spam-Status: No, hits=-99.5 required=10.0 tests=ALL_TRUSTED,NO_REAL_NAME X-Spam-Check-By: apache.org Received: from [140.211.11.3] (HELO eris.apache.org) (140.211.11.3) by apache.org (qpsmtpd/0.29) with ESMTP; Fri, 22 Jun 2007 15:48:08 -0700 Received: by eris.apache.org (Postfix, from userid 65534) id 6931C1A981A; Fri, 22 Jun 2007 15:47:48 -0700 (PDT) Content-Type: text/plain; charset="utf-8" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit Subject: svn commit: r549975 - in /lucene/hadoop/trunk: ./ src/java/org/apache/hadoop/dfs/ src/java/org/apache/hadoop/fs/ src/test/org/apache/hadoop/dfs/ Date: Fri, 22 Jun 2007 22:47:48 -0000 To: hadoop-commits@lucene.apache.org From: cutting@apache.org X-Mailer: svnmailer-1.1.0 Message-Id: <20070622224748.6931C1A981A@eris.apache.org> X-Virus-Checked: Checked by ClamAV on apache.org Author: cutting Date: Fri Jun 22 15:47:47 2007 New Revision: 549975 URL: http://svn.apache.org/viewvc?view=rev&rev=549975 Log: HADOOP-1292. Change 'bin/hadoop fs -get' to first copy files to a temporary name, then rename them to their final name, so that failures don't leave partial files. Contributed by Tsz Wo Sze. Modified: lucene/hadoop/trunk/CHANGES.txt lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/DistributedFileSystem.java lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/ChecksumFileSystem.java lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/FileUtil.java lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/FsShell.java lucene/hadoop/trunk/src/test/org/apache/hadoop/dfs/TestDFSShell.java Modified: lucene/hadoop/trunk/CHANGES.txt URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/CHANGES.txt?view=diff&rev=549975&r1=549974&r2=549975 ============================================================================== --- lucene/hadoop/trunk/CHANGES.txt (original) +++ lucene/hadoop/trunk/CHANGES.txt Fri Jun 22 15:47:47 2007 @@ -242,6 +242,10 @@ 74. HADOOP-1518. Add a session id to job metrics, for use by HOD. (David Bowen via cutting) + 75. HADOOP-1292. Change 'bin/hadoop fs -get' to first copy files to + a temporary name, then rename them to their final name, so that + failures don't leave partial files. (Tsz Wo Sze via cutting) + Release 0.13.0 - 2007-06-08 Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/DistributedFileSystem.java URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/DistributedFileSystem.java?view=diff&rev=549975&r1=549974&r2=549975 ============================================================================== --- lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/DistributedFileSystem.java (original) +++ lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/DistributedFileSystem.java Fri Jun 22 15:47:47 2007 @@ -65,7 +65,7 @@ int port = uri.getPort(); this.dfs = new DFSClient(new InetSocketAddress(host, port), conf); this.uri = URI.create("hdfs://"+host+":"+port); - this.localFs = getNamed("file:///", conf); + this.localFs = getLocal(conf); } public Path getWorkingDirectory() { Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/ChecksumFileSystem.java URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/ChecksumFileSystem.java?view=diff&rev=549975&r1=549974&r2=549975 ============================================================================== --- lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/ChecksumFileSystem.java (original) +++ lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/ChecksumFileSystem.java Fri Jun 22 15:47:47 2007 @@ -567,8 +567,8 @@ @Override public void copyFromLocalFile(boolean delSrc, Path src, Path dst) throws IOException { - FileSystem localFs = getNamed("file:///", getConf()); - FileUtil.copy(localFs, src, this, dst, delSrc, getConf()); + Configuration conf = getConf(); + FileUtil.copy(getLocal(conf), src, this, dst, delSrc, conf); } /** @@ -578,8 +578,8 @@ @Override public void copyToLocalFile(boolean delSrc, Path src, Path dst) throws IOException { - FileSystem localFs = getNamed("file:///", getConf()); - FileUtil.copy(this, src, localFs, dst, delSrc, getConf()); + Configuration conf = getConf(); + FileUtil.copy(this, src, getLocal(conf), dst, delSrc, conf); } /** @@ -592,7 +592,7 @@ throws IOException { if (!fs.isDirectory(src)) { // source is a file fs.copyToLocalFile(src, dst); - FileSystem localFs = getNamed("file:///", getConf()); + FileSystem localFs = getLocal(getConf()); if (localFs instanceof ChecksumFileSystem) { localFs = ((ChecksumFileSystem) localFs).getRawFileSystem(); } Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/FileUtil.java URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/FileUtil.java?view=diff&rev=549975&r1=549974&r2=549975 ============================================================================== --- lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/FileUtil.java (original) +++ lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/FileUtil.java Fri Jun 22 15:47:47 2007 @@ -194,9 +194,6 @@ public static boolean copy(FileSystem srcFS, Path src, File dst, boolean deleteSource, Configuration conf) throws IOException { - - dst = checkDest(src.getName(), dst); - if (srcFS.isDirectory(src)) { if (!dst.mkdirs()) { return false; @@ -431,5 +428,27 @@ String cmd = "chmod " + perm + " " + filename; Process p = Runtime.getRuntime().exec(cmd, null); return p.waitFor(); + } + + /** + * Create a tmp file for a base file. + * @param basefile the base file of the tmp + * @param prefix file name prefix of tmp + * @param isDeleteOnExit if true, the tmp will be deleted when the VM exits + * @return a newly created tmp file + * @exception IOException If a tmp file cannot created + * @see java.io.File#createTempFile(String, String, File) + * @see java.io.File#deleteOnExit() + */ + public static final File createLocalTempFile(final File basefile, + final String prefix, + final boolean isDeleteOnExit) + throws IOException { + File tmp = File.createTempFile(prefix + basefile.getName(), + "", basefile.getParentFile()); + if (isDeleteOnExit) { + tmp.deleteOnExit(); + } + return tmp; } } Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/FsShell.java URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/FsShell.java?view=diff&rev=549975&r1=549974&r2=549975 ============================================================================== --- lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/FsShell.java (original) +++ lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/FsShell.java Fri Jun 22 15:47:47 2007 @@ -21,7 +21,6 @@ import java.util.*; import org.apache.hadoop.conf.*; -import org.apache.hadoop.dfs.DistributedFileSystem; import org.apache.hadoop.ipc.*; import org.apache.hadoop.util.ToolBase; @@ -138,18 +137,86 @@ } cat(srcf); } else { - Path [] srcs = fs.globPaths(new Path(srcf)); - if (srcs.length > 1 && !new File(dstf).isDirectory()) { - throw new IOException("When copying multiple files, " - + "destination should be a directory."); + copyToLocal(fs, new Path(srcf), new File(dstf), copyCrc); + } + } + + /** + * The prefix for the tmp file used in copyToLocal. + * It must be at least three characters long, required by + * {@link java.io.File#createTempFile(String, String, File)}. + */ + static final String COPYTOLOCAL_PREFIX = "_copyToLocal_"; + + /** + * Copy a source file from a given file system to local destination. + * @param srcFS source file system + * @param src source path + * @param dst destination + * @param copyCrc copy CRC files? + * @exception IOException If some IO failed + */ + private void copyToLocal(final FileSystem srcFS, final Path src, + final File dst, final boolean copyCrc) + throws IOException { + if (srcFS.isDirectory(src)) { //src is a directory + dst.mkdir(); + if (!dst.isDirectory()) { + throw new IOException("cannot create directory for local destination \"" + + dst + "\"."); + } + for(Path p : srcFS.listPaths(src)) { + copyToLocal(srcFS, p, + srcFS.isDirectory(p)? new File(dst, p.getName()): dst, copyCrc); } - Path dst = new Path(dstf); - for(int i=0; i