2727import  org .apache .hadoop .fs .FSDataInputStream ;
2828import  org .apache .hadoop .fs .FileSystem ;
2929import  org .apache .hadoop .fs .Path ;
30+ import  org .apache .hadoop .io .SequenceFile .CompressionType ;
3031import  org .apache .hadoop .test .GenericTestUtils ;
3132import  org .junit .Test ;
3233
34+ /** Tests sync based seek reads/write intervals inside SequenceFiles. */ 
3335public  class  TestSequenceFileSync  {
3436  private  static  final  int  NUMRECORDS  = 2000 ;
3537  private  static  final  int  RECORDSIZE  = 80 ;
36-   private  static  final  Random  rand  = new  Random ();
38+   private  static  final  Random  RAND  = new  Random ();
3739
3840  private  final  static  String  REC_FMT  = "%d RECORDID %d : " ;
3941
@@ -46,37 +48,110 @@ private static void forOffset(SequenceFile.Reader reader,
4648    reader .next (key , val );
4749    assertEquals (key .get (), expectedRecord );
4850    final  String  test  = String .format (REC_FMT , expectedRecord , expectedRecord );
49-     assertEquals ("Invalid value "  + val , 0 , val .find (test , 0 ));
51+     assertEquals (
52+         "Invalid value in iter "  + iter  + ": "  + val ,
53+         0 ,
54+         val .find (test , 0 ));
55+   }
56+ 
57+   @ Test 
58+   public  void  testDefaultSyncInterval () throws  IOException  {
59+     // Uses the default sync interval of 100 KB 
60+     final  Configuration  conf  = new  Configuration ();
61+     final  FileSystem  fs  = FileSystem .getLocal (conf );
62+     final  Path  path  = new  Path (GenericTestUtils .getTempPath (
63+             "sequencefile.sync.test" ));
64+     final  IntWritable  input  = new  IntWritable ();
65+     final  Text  val  = new  Text ();
66+     SequenceFile .Writer  writer  = new  SequenceFile .Writer (
67+         conf ,
68+         SequenceFile .Writer .file (path ),
69+         SequenceFile .Writer .compression (CompressionType .NONE ),
70+         SequenceFile .Writer .keyClass (IntWritable .class ),
71+         SequenceFile .Writer .valueClass (Text .class )
72+     );
73+     try  {
74+       writeSequenceFile (writer , NUMRECORDS *4 );
75+       for  (int  i  = 0 ; i  < 5 ; i ++) {
76+         final  SequenceFile .Reader  reader ;
77+ 
78+         //try different SequenceFile.Reader constructors 
79+         if  (i  % 2  == 0 ) {
80+           final  int  buffersize  = conf .getInt ("io.file.buffer.size" , 4096 );
81+           reader  = new  SequenceFile .Reader (conf ,
82+               SequenceFile .Reader .file (path ),
83+               SequenceFile .Reader .bufferSize (buffersize ));
84+         } else  {
85+           final  FSDataInputStream  in  = fs .open (path );
86+           final  long  length  = fs .getFileStatus (path ).getLen ();
87+           reader  = new  SequenceFile .Reader (conf ,
88+               SequenceFile .Reader .stream (in ),
89+               SequenceFile .Reader .start (0L ),
90+               SequenceFile .Reader .length (length ));
91+         }
92+ 
93+         try  {
94+           forOffset (reader , input , val , i , 0 , 0 );
95+           forOffset (reader , input , val , i , 65 , 0 );
96+           // There would be over 1000 records within 
97+           // this sync interval 
98+           forOffset (reader , input , val , i , 2000 , 1101 );
99+           forOffset (reader , input , val , i , 0 , 0 );
100+         } finally  {
101+           reader .close ();
102+         }
103+       }
104+     } finally  {
105+       fs .delete (path , false );
106+     }
50107  }
51108
52109  @ Test 
53110  public  void  testLowSyncpoint () throws  IOException  {
111+     // Uses a smaller sync interval of 2000 bytes 
54112    final  Configuration  conf  = new  Configuration ();
55113    final  FileSystem  fs  = FileSystem .getLocal (conf );
56114    final  Path  path  = new  Path (GenericTestUtils .getTempPath (
57115        "sequencefile.sync.test" ));
58116    final  IntWritable  input  = new  IntWritable ();
59117    final  Text  val  = new  Text ();
60-     SequenceFile .Writer  writer  = new  SequenceFile .Writer (fs , conf , path ,
61-         IntWritable .class , Text .class );
118+     SequenceFile .Writer  writer  = new  SequenceFile .Writer (
119+         conf ,
120+         SequenceFile .Writer .file (path ),
121+         SequenceFile .Writer .compression (CompressionType .NONE ),
122+         SequenceFile .Writer .keyClass (IntWritable .class ),
123+         SequenceFile .Writer .valueClass (Text .class ),
124+         SequenceFile .Writer .syncInterval (20 *100 )
125+     );
126+     // Ensure the custom sync interval value is set 
127+     assertEquals (writer .syncInterval , 20 *100 );
62128    try  {
63129      writeSequenceFile (writer , NUMRECORDS );
64-       for  (int  i  = 0 ; i  < 5   ; i ++) {
65-        final  SequenceFile .Reader  reader ;
130+       for  (int  i  = 0 ; i  < 5 ; i ++) {
131+          final  SequenceFile .Reader  reader ;
66132
67-        //try different SequenceFile.Reader constructors 
68-        if  (i  % 2  == 0 ) {
69-          reader  = new  SequenceFile .Reader (fs , path , conf );
70-        } else  {
71-          final  FSDataInputStream  in  = fs .open (path );
72-          final  long  length  = fs .getFileStatus (path ).getLen ();
73-          final  int  buffersize  = conf .getInt ("io.file.buffer.size" , 4096 );
74-          reader  = new  SequenceFile .Reader (in , buffersize , 0L , length , conf );
75-        }
133+         //try different SequenceFile.Reader constructors 
134+         if  (i  % 2  == 0 ) {
135+           final  int  bufferSize  = conf .getInt ("io.file.buffer.size" , 4096 );
136+           reader  = new  SequenceFile .Reader (
137+               conf ,
138+               SequenceFile .Reader .file (path ),
139+               SequenceFile .Reader .bufferSize (bufferSize ));
140+         } else  {
141+           final  FSDataInputStream  in  = fs .open (path );
142+           final  long  length  = fs .getFileStatus (path ).getLen ();
143+           reader  = new  SequenceFile .Reader (
144+               conf ,
145+               SequenceFile .Reader .stream (in ),
146+               SequenceFile .Reader .start (0L ),
147+               SequenceFile .Reader .length (length ));
148+         }
76149
77-        try  {
150+          try  {
78151          forOffset (reader , input , val , i , 0 , 0 );
79152          forOffset (reader , input , val , i , 65 , 0 );
153+           // There would be only a few records within 
154+           // this sync interval 
80155          forOffset (reader , input , val , i , 2000 , 21 );
81156          forOffset (reader , input , val , i , 0 , 0 );
82157        } finally  {
@@ -88,7 +163,7 @@ public void testLowSyncpoint() throws IOException {
88163    }
89164  }
90165
91-   public  static  void  writeSequenceFile (SequenceFile .Writer  writer ,
166+   private  static  void  writeSequenceFile (SequenceFile .Writer  writer ,
92167      int  numRecords ) throws  IOException  {
93168    final  IntWritable  key  = new  IntWritable ();
94169    final  Text  val  = new  Text ();
@@ -100,13 +175,13 @@ public static void writeSequenceFile(SequenceFile.Writer writer,
100175    writer .close ();
101176  }
102177
103-   static  void  randomText (Text  val , int  id , int  recordSize ) {
178+   private   static  void  randomText (Text  val , int  id , int  recordSize ) {
104179    val .clear ();
105180    final  StringBuilder  ret  = new  StringBuilder (recordSize );
106181    ret .append (String .format (REC_FMT , id , id ));
107182    recordSize  -= ret .length ();
108183    for  (int  i  = 0 ; i  < recordSize ; ++i ) {
109-       ret .append (rand .nextInt (9 ));
184+       ret .append (RAND .nextInt (9 ));
110185    }
111186    val .set (ret .toString ());
112187  }
0 commit comments