Return-Path: Delivered-To: apmail-lucene-hadoop-commits-archive@locus.apache.org Received: (qmail 24132 invoked from network); 20 Dec 2006 19:29:17 -0000 Received: from hermes.apache.org (HELO mail.apache.org) (140.211.11.2) by minotaur.apache.org with SMTP; 20 Dec 2006 19:29:17 -0000 Received: (qmail 89249 invoked by uid 500); 20 Dec 2006 19:29:24 -0000 Delivered-To: apmail-lucene-hadoop-commits-archive@lucene.apache.org Received: (qmail 89167 invoked by uid 500); 20 Dec 2006 19:29:24 -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 89158 invoked by uid 99); 20 Dec 2006 19:29:24 -0000 Received: from herse.apache.org (HELO herse.apache.org) (140.211.11.133) by apache.org (qpsmtpd/0.29) with ESMTP; Wed, 20 Dec 2006 11:29:24 -0800 X-ASF-Spam-Status: No, hits=-9.4 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; Wed, 20 Dec 2006 11:29:16 -0800 Received: by eris.apache.org (Postfix, from userid 65534) id 235061A981A; Wed, 20 Dec 2006 11:28:27 -0800 (PST) Content-Type: text/plain; charset="utf-8" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit Subject: svn commit: r489175 - in /lucene/hadoop/trunk: CHANGES.txt src/java/org/apache/hadoop/fs/FsShell.java Date: Wed, 20 Dec 2006 19:28:27 -0000 To: hadoop-commits@lucene.apache.org From: cutting@apache.org X-Mailer: svnmailer-1.1.0 Message-Id: <20061220192827.235061A981A@eris.apache.org> X-Virus-Checked: Checked by ClamAV on apache.org Author: cutting Date: Wed Dec 20 11:28:26 2006 New Revision: 489175 URL: http://svn.apache.org/viewvc?view=rev&rev=489175 Log: HADOOP-738. Extend FsShell '-put' and '-get' commands to accept standard input and output, respectively. Contributed by Wendy. Modified: lucene/hadoop/trunk/CHANGES.txt lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/FsShell.java Modified: lucene/hadoop/trunk/CHANGES.txt URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/CHANGES.txt?view=diff&rev=489175&r1=489174&r2=489175 ============================================================================== --- lucene/hadoop/trunk/CHANGES.txt (original) +++ lucene/hadoop/trunk/CHANGES.txt Wed Dec 20 11:28:26 2006 @@ -130,6 +130,10 @@ 36. HADOOP-814. Optimize locking in namenode. (Dhruba Borthakur via cutting) +37. HADOOP-738. Change 'fs -put' and 'fs -get' commands to accept + standard input and output, respectively. Standard i/o is + specified by a file named '-'. (Wendy Chien via cutting) + Release 0.9.2 - 2006-12-15 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=489175&r1=489174&r2=489175 ============================================================================== --- lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/FsShell.java (original) +++ lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/FsShell.java Wed Dec 20 11:28:26 2006 @@ -38,11 +38,39 @@ conf.setQuietMode(true); this.fs = FileSystem.get(conf); } + + /** + * Copies from stdin to the indicated file. + */ + private void copyFromStdin(Path dst) throws IOException { + if (fs.isDirectory(dst)) { + throw new IOException("When source is stdin, destination must be a file."); + } + if (fs.exists(dst)) { + throw new IOException("Target " + dst.toString() + " already exists."); + } + FSDataOutputStream out = fs.create(dst); + byte buf[] = new byte[conf.getInt("io.file.buffer.size", 4096)]; + try { + int bytesRead = System.in.read(buf); + while (bytesRead >= 0) { + out.write(buf, 0, bytesRead); + bytesRead = System.in.read(buf); + } + } finally { + out.close(); + } + } + /** * Add a local file to the indicated FileSystem name. src is kept. */ void copyFromLocal(Path src, String dstf) throws IOException { + if (src.toString().equals("-")) { + copyFromStdin(new Path(dstf)); + } else { fs.copyFromLocalFile(src, new Path(dstf)); + } } /** @@ -74,14 +102,21 @@ } String srcf = argv[pos++]; String dstf = argv[pos++]; - Path [] srcs = fs.globPaths( new Path(srcf) ); - if( srcs.length > 1 && !new File( dstf ).isDirectory()) { - throw new IOException( "When copy multiple files, " - + "destination should be a directory." ); - } - Path dst = new Path( dstf ); - for( int i=0; i 1 && !new File( dstf ).isDirectory()) { + throw new IOException( "When copying multiple files, " + + "destination should be a directory." ); + } + Path dst = new Path( dstf ); + for( int i=0; i