HDFS-206. Support for head in FSShell. Contributed by Gabor Bota.
This commit is contained in:
parent
2fae63aa60
commit
81225430d0
@ -64,6 +64,7 @@ public static void registerCommands(CommandFactory factory) {
|
|||||||
factory.registerCommands(SetReplication.class);
|
factory.registerCommands(SetReplication.class);
|
||||||
factory.registerCommands(Stat.class);
|
factory.registerCommands(Stat.class);
|
||||||
factory.registerCommands(Tail.class);
|
factory.registerCommands(Tail.class);
|
||||||
|
factory.registerCommands(Head.class);
|
||||||
factory.registerCommands(Test.class);
|
factory.registerCommands(Test.class);
|
||||||
factory.registerCommands(Touch.class);
|
factory.registerCommands(Touch.class);
|
||||||
factory.registerCommands(Truncate.class);
|
factory.registerCommands(Truncate.class);
|
||||||
|
@ -0,0 +1,78 @@
|
|||||||
|
/**
|
||||||
|
* 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.shell;
|
||||||
|
|
||||||
|
import org.apache.hadoop.classification.InterfaceAudience;
|
||||||
|
import org.apache.hadoop.classification.InterfaceStability;
|
||||||
|
import org.apache.hadoop.fs.FSDataInputStream;
|
||||||
|
import org.apache.hadoop.fs.PathIsDirectoryException;
|
||||||
|
import org.apache.hadoop.io.IOUtils;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.util.LinkedList;
|
||||||
|
import java.util.List;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Show the first 1KB of the file.
|
||||||
|
*/
|
||||||
|
@InterfaceAudience.Private
|
||||||
|
@InterfaceStability.Unstable
|
||||||
|
|
||||||
|
class Head extends FsCommand {
|
||||||
|
public static void registerCommands(CommandFactory factory) {
|
||||||
|
factory.addClass(Head.class, "-head");
|
||||||
|
}
|
||||||
|
public static final String NAME = "head";
|
||||||
|
public static final String USAGE = "<file>";
|
||||||
|
public static final String DESCRIPTION =
|
||||||
|
"Show the first 1KB of the file.\n";
|
||||||
|
|
||||||
|
private long endingOffset = 1024;
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected void processOptions(LinkedList<String> args) throws IOException {
|
||||||
|
CommandFormat cf = new CommandFormat(1, 1);
|
||||||
|
cf.parse(args);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected List<PathData> expandArgument(String arg) throws IOException {
|
||||||
|
List<PathData> items = new LinkedList<PathData>();
|
||||||
|
items.add(new PathData(arg, getConf()));
|
||||||
|
return items;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected void processPath(PathData item) throws IOException {
|
||||||
|
if (item.stat.isDirectory()) {
|
||||||
|
throw new PathIsDirectoryException(item.toString());
|
||||||
|
}
|
||||||
|
|
||||||
|
dumpToOffset(item);
|
||||||
|
}
|
||||||
|
|
||||||
|
private void dumpToOffset(PathData item) throws IOException {
|
||||||
|
FSDataInputStream in = item.fs.open(item.path);
|
||||||
|
try {
|
||||||
|
IOUtils.copyBytes(in, System.out, endingOffset, false);
|
||||||
|
} finally {
|
||||||
|
in.close();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
@ -400,6 +400,19 @@ Exit Code:
|
|||||||
|
|
||||||
Returns 0 on success and non-zero on error.
|
Returns 0 on success and non-zero on error.
|
||||||
|
|
||||||
|
head
|
||||||
|
----
|
||||||
|
|
||||||
|
Usage: `hadoop fs -head URI`
|
||||||
|
|
||||||
|
Displays first kilobyte of the file to stdout.
|
||||||
|
|
||||||
|
Example:
|
||||||
|
|
||||||
|
* `hadoop fs -head pathname`
|
||||||
|
|
||||||
|
Exit Code: Returns 0 on success and -1 on error.
|
||||||
|
|
||||||
help
|
help
|
||||||
----
|
----
|
||||||
|
|
||||||
|
@ -892,6 +892,33 @@ public void testURIPaths() throws Exception {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Test that -head displays first kilobyte of the file to stdout.
|
||||||
|
*/
|
||||||
|
@Test (timeout = 30000)
|
||||||
|
public void testHead() throws Exception {
|
||||||
|
final int fileLen = 5 * BLOCK_SIZE;
|
||||||
|
|
||||||
|
// create a text file with multiple KB bytes (and multiple blocks)
|
||||||
|
final Path testFile = new Path("testHead", "file1");
|
||||||
|
final String text = RandomStringUtils.randomAscii(fileLen);
|
||||||
|
try (OutputStream pout = dfs.create(testFile)) {
|
||||||
|
pout.write(text.getBytes());
|
||||||
|
}
|
||||||
|
final ByteArrayOutputStream out = new ByteArrayOutputStream();
|
||||||
|
System.setOut(new PrintStream(out));
|
||||||
|
final String[] argv = new String[]{"-head", testFile.toString()};
|
||||||
|
final int ret = ToolRunner.run(new FsShell(dfs.getConf()), argv);
|
||||||
|
|
||||||
|
assertEquals(Arrays.toString(argv) + " returned " + ret, 0, ret);
|
||||||
|
assertEquals("-head returned " + out.size() + " bytes data, expected 1KB",
|
||||||
|
1024, out.size());
|
||||||
|
// tailed out last 1KB of the file content
|
||||||
|
assertArrayEquals("Head output doesn't match input",
|
||||||
|
text.substring(0, 1024).getBytes(), out.toByteArray());
|
||||||
|
out.reset();
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Test that -tail displays last kilobyte of the file to stdout.
|
* Test that -tail displays last kilobyte of the file to stdout.
|
||||||
*/
|
*/
|
||||||
|
Loading…
Reference in New Issue
Block a user