3131
3232import org .apache .commons .io .IOUtils ;
3333import org .apache .hadoop .conf .Configuration ;
34+ import org .apache .hadoop .fs .Path ;
35+ import org .apache .hadoop .io .SequenceFile ;
3436import org .apache .hadoop .test .GenericTestUtils ;
3537import org .junit .Test ;
3638
@@ -46,22 +48,19 @@ public class TestTextCommand {
4648 private static final String TEXT_FILENAME =
4749 new File (TEST_ROOT_DIR , "testtextfile.txt" ).toURI ().getPath ();
4850
51+ private static final String SEPARATOR = System .getProperty ("line.separator" );
52+
4953 /**
5054 * Tests whether binary Avro data files are displayed correctly.
5155 */
5256 @ Test (timeout = 30000 )
5357 public void testDisplayForAvroFiles () throws Exception {
5458 String expectedOutput =
55- "{\" station\" :\" 011990-99999\" ,\" time\" :-619524000000,\" temp\" :0}" +
56- System .getProperty ("line.separator" ) +
57- "{\" station\" :\" 011990-99999\" ,\" time\" :-619506000000,\" temp\" :22}" +
58- System .getProperty ("line.separator" ) +
59- "{\" station\" :\" 011990-99999\" ,\" time\" :-619484400000,\" temp\" :-11}" +
60- System .getProperty ("line.separator" ) +
61- "{\" station\" :\" 012650-99999\" ,\" time\" :-655531200000,\" temp\" :111}" +
62- System .getProperty ("line.separator" ) +
63- "{\" station\" :\" 012650-99999\" ,\" time\" :-655509600000,\" temp\" :78}" +
64- System .getProperty ("line.separator" );
59+ "{\" station\" :\" 011990-99999\" ,\" time\" :-619524000000,\" temp\" :0}" + SEPARATOR
60+ + "{\" station\" :\" 011990-99999\" ,\" time\" :-619506000000,\" temp\" :22}" + SEPARATOR
61+ + "{\" station\" :\" 011990-99999\" ,\" time\" :-619484400000,\" temp\" :-11}" + SEPARATOR
62+ + "{\" station\" :\" 012650-99999\" ,\" time\" :-655531200000,\" temp\" :111}" + SEPARATOR
63+ + "{\" station\" :\" 012650-99999\" ,\" time\" :-655509600000,\" temp\" :78}" + SEPARATOR ;
6564
6665 String output = readUsingTextCommand (AVRO_FILENAME ,
6766 generateWeatherAvroBinaryData ());
@@ -104,19 +103,24 @@ private String readUsingTextCommand(String fileName, byte[] fileContents)
104103 throws Exception {
105104 createFile (fileName , fileContents );
106105
107- // Prepare and call the Text command's protected getInputStream method
108- // using reflection.
109106 Configuration conf = new Configuration ();
110107 URI localPath = new URI (fileName );
111- PathData pathData = new PathData (localPath , conf );
108+ return readUsingTextCommand (localPath , conf );
109+ }
110+ // Read a file using Display.Text class.
111+ private String readUsingTextCommand (URI uri , Configuration conf )
112+ throws Exception {
113+ // Prepare and call the Text command's protected getInputStream method
114+ // using reflection.
115+ PathData pathData = new PathData (uri , conf );
112116 Display .Text text = new Display .Text () {
113117 @ Override
114118 public InputStream getInputStream (PathData item ) throws IOException {
115119 return super .getInputStream (item );
116120 }
117121 };
118122 text .setConf (conf );
119- InputStream stream = ( InputStream ) text .getInputStream (pathData );
123+ InputStream stream = text .getInputStream (pathData );
120124 return inputStreamToString (stream );
121125 }
122126
@@ -232,5 +236,21 @@ private byte[] generateWeatherAvroBinaryData() {
232236
233237 return contents ;
234238 }
239+
240+ @ Test
241+ public void testDisplayForNonWritableSequenceFile () throws Exception {
242+ Configuration conf = new Configuration ();
243+ conf .set ("io.serializations" , "org.apache.hadoop.io.serializer.JavaSerialization" );
244+ Path path = new Path (String .valueOf (TEST_ROOT_DIR ), "NonWritableSequenceFile" );
245+ SequenceFile .Writer writer = SequenceFile .createWriter (conf , SequenceFile .Writer .file (path ),
246+ SequenceFile .Writer .keyClass (String .class ), SequenceFile .Writer .valueClass (String .class ));
247+ writer .append ("Key1" , "Value1" );
248+ writer .append ("Key2" , "Value2" );
249+ writer .close ();
250+ String expected = "Key1\t Value1" + SEPARATOR + "Key2\t Value2" + SEPARATOR ;
251+ URI uri = path .toUri ();
252+ System .out .println (expected );
253+ assertEquals (expected , readUsingTextCommand (uri , conf ));
254+ }
235255}
236256
0 commit comments