HADOOP-8728. Display (fs -text) shouldn't hard-depend on Writable serialized sequence files. (#5010)

Co-authored-by: Ashutosh Gupta <ashugpt@amazon.com>
Signed-off-by: Akira Ajisaka <aajisaka@apache.org>
This commit is contained in:
Ashutosh Gupta 2022-11-21 05:54:50 +00:00 committed by GitHub
parent 2e993fdf4e
commit 696d042054
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
2 changed files with 42 additions and 22 deletions

View File

@ -47,7 +47,6 @@
import org.apache.hadoop.io.DataOutputBuffer; import org.apache.hadoop.io.DataOutputBuffer;
import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.io.SequenceFile; import org.apache.hadoop.io.SequenceFile;
import org.apache.hadoop.io.Writable;
import org.apache.hadoop.io.compress.CompressionCodec; import org.apache.hadoop.io.compress.CompressionCodec;
import org.apache.hadoop.io.compress.CompressionCodecFactory; import org.apache.hadoop.io.compress.CompressionCodecFactory;
import org.apache.hadoop.util.ReflectionUtils; import org.apache.hadoop.util.ReflectionUtils;
@ -217,8 +216,8 @@ protected void processPath(PathData item) throws IOException {
protected class TextRecordInputStream extends InputStream { protected class TextRecordInputStream extends InputStream {
SequenceFile.Reader r; SequenceFile.Reader r;
Writable key; Object key;
Writable val; Object val;
DataInputBuffer inbuf; DataInputBuffer inbuf;
DataOutputBuffer outbuf; DataOutputBuffer outbuf;
@ -228,10 +227,8 @@ public TextRecordInputStream(FileStatus f) throws IOException {
final Configuration lconf = getConf(); final Configuration lconf = getConf();
r = new SequenceFile.Reader(lconf, r = new SequenceFile.Reader(lconf,
SequenceFile.Reader.file(fpath)); SequenceFile.Reader.file(fpath));
key = ReflectionUtils.newInstance( key = ReflectionUtils.newInstance(r.getKeyClass(), lconf);
r.getKeyClass().asSubclass(Writable.class), lconf); val = ReflectionUtils.newInstance(r.getValueClass(), lconf);
val = ReflectionUtils.newInstance(
r.getValueClass().asSubclass(Writable.class), lconf);
inbuf = new DataInputBuffer(); inbuf = new DataInputBuffer();
outbuf = new DataOutputBuffer(); outbuf = new DataOutputBuffer();
} }
@ -240,8 +237,11 @@ public TextRecordInputStream(FileStatus f) throws IOException {
public int read() throws IOException { public int read() throws IOException {
int ret; int ret;
if (null == inbuf || -1 == (ret = inbuf.read())) { if (null == inbuf || -1 == (ret = inbuf.read())) {
if (!r.next(key, val)) { key = r.next(key);
if (key == null) {
return -1; return -1;
} else {
val = r.getCurrentValue(val);
} }
byte[] tmp = key.toString().getBytes(StandardCharsets.UTF_8); byte[] tmp = key.toString().getBytes(StandardCharsets.UTF_8);
outbuf.write(tmp, 0, tmp.length); outbuf.write(tmp, 0, tmp.length);

View File

@ -31,6 +31,8 @@
import org.apache.commons.io.IOUtils; import org.apache.commons.io.IOUtils;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.SequenceFile;
import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.test.GenericTestUtils;
import org.junit.Test; import org.junit.Test;
@ -46,22 +48,19 @@ public class TestTextCommand {
private static final String TEXT_FILENAME = private static final String TEXT_FILENAME =
new File(TEST_ROOT_DIR, "testtextfile.txt").toURI().getPath(); new File(TEST_ROOT_DIR, "testtextfile.txt").toURI().getPath();
private static final String SEPARATOR = System.getProperty("line.separator");
/** /**
* Tests whether binary Avro data files are displayed correctly. * Tests whether binary Avro data files are displayed correctly.
*/ */
@Test (timeout = 30000) @Test (timeout = 30000)
public void testDisplayForAvroFiles() throws Exception { public void testDisplayForAvroFiles() throws Exception {
String expectedOutput = String expectedOutput =
"{\"station\":\"011990-99999\",\"time\":-619524000000,\"temp\":0}" + "{\"station\":\"011990-99999\",\"time\":-619524000000,\"temp\":0}" + SEPARATOR
System.getProperty("line.separator") + + "{\"station\":\"011990-99999\",\"time\":-619506000000,\"temp\":22}" + SEPARATOR
"{\"station\":\"011990-99999\",\"time\":-619506000000,\"temp\":22}" + + "{\"station\":\"011990-99999\",\"time\":-619484400000,\"temp\":-11}" + SEPARATOR
System.getProperty("line.separator") + + "{\"station\":\"012650-99999\",\"time\":-655531200000,\"temp\":111}" + SEPARATOR
"{\"station\":\"011990-99999\",\"time\":-619484400000,\"temp\":-11}" + + "{\"station\":\"012650-99999\",\"time\":-655509600000,\"temp\":78}" + SEPARATOR;
System.getProperty("line.separator") +
"{\"station\":\"012650-99999\",\"time\":-655531200000,\"temp\":111}" +
System.getProperty("line.separator") +
"{\"station\":\"012650-99999\",\"time\":-655509600000,\"temp\":78}" +
System.getProperty("line.separator");
String output = readUsingTextCommand(AVRO_FILENAME, String output = readUsingTextCommand(AVRO_FILENAME,
generateWeatherAvroBinaryData()); generateWeatherAvroBinaryData());
@ -104,11 +103,16 @@ private String readUsingTextCommand(String fileName, byte[] fileContents)
throws Exception { throws Exception {
createFile(fileName, fileContents); createFile(fileName, fileContents);
// Prepare and call the Text command's protected getInputStream method
// using reflection.
Configuration conf = new Configuration(); Configuration conf = new Configuration();
URI localPath = new URI(fileName); URI localPath = new URI(fileName);
PathData pathData = new PathData(localPath, conf); return readUsingTextCommand(localPath, conf);
}
// Read a file using Display.Text class.
private String readUsingTextCommand(URI uri, Configuration conf)
throws Exception {
// Prepare and call the Text command's protected getInputStream method
// using reflection.
PathData pathData = new PathData(uri, conf);
Display.Text text = new Display.Text() { Display.Text text = new Display.Text() {
@Override @Override
public InputStream getInputStream(PathData item) throws IOException { public InputStream getInputStream(PathData item) throws IOException {
@ -116,7 +120,7 @@ public InputStream getInputStream(PathData item) throws IOException {
} }
}; };
text.setConf(conf); text.setConf(conf);
InputStream stream = (InputStream) text.getInputStream(pathData); InputStream stream = text.getInputStream(pathData);
return inputStreamToString(stream); return inputStreamToString(stream);
} }
@ -232,5 +236,21 @@ private byte[] generateWeatherAvroBinaryData() {
return contents; return contents;
} }
@Test
public void testDisplayForNonWritableSequenceFile() throws Exception {
Configuration conf = new Configuration();
conf.set("io.serializations", "org.apache.hadoop.io.serializer.JavaSerialization");
Path path = new Path(String.valueOf(TEST_ROOT_DIR), "NonWritableSequenceFile");
SequenceFile.Writer writer = SequenceFile.createWriter(conf, SequenceFile.Writer.file(path),
SequenceFile.Writer.keyClass(String.class), SequenceFile.Writer.valueClass(String.class));
writer.append("Key1", "Value1");
writer.append("Key2", "Value2");
writer.close();
String expected = "Key1\tValue1" + SEPARATOR + "Key2\tValue2" + SEPARATOR;
URI uri = path.toUri();
System.out.println(expected);
assertEquals(expected, readUsingTextCommand(uri, conf));
}
} }