19
19
20
20
public class SequenceFileReader extends AbstractFileReader <SequenceFileReader .SequenceRecord <Writable , Writable >> {
21
21
22
+ public static final String FIELD_NAME_KEY_DEFAULT = "key" ;
23
+ public static final String FIELD_NAME_VALUE_DEFAULT = "value" ;
24
+
22
25
private static final int DEFAULT_BUFFER_SIZE = 4096 ;
23
26
private static final String FILE_READER_SEQUENCE = FILE_READER_PREFIX + "sequence." ;
27
+ private static final String FILE_READER_SEQUENCE_FIELD_NAME_PREFIX = FILE_READER_SEQUENCE + "field_name." ;
28
+
24
29
public static final String FILE_READER_BUFFER_SIZE = FILE_READER_SEQUENCE + "buffer_size" ;
30
+ public static final String FILE_READER_SEQUENCE_FIELD_NAME_KEY = FILE_READER_SEQUENCE_FIELD_NAME_PREFIX + "key" ;
31
+ public static final String FILE_READER_SEQUENCE_FIELD_NAME_VALUE = FILE_READER_SEQUENCE_FIELD_NAME_PREFIX + "value" ;
25
32
26
- private static final String FIELD_KEY = "key" ;
27
- private static final String FIELD_VALUE = "value" ;
28
33
29
34
private final SequenceFile .Reader reader ;
30
35
private final Writable key , value ;
31
- private final Schema schema ;
32
36
private final SeqOffset offset ;
37
+ private final Schema schema ;
38
+ private String keyFieldName , valueFieldName ;
33
39
private long recordIndex , hasNextIndex ;
34
40
private boolean hasNext ;
35
41
@@ -42,14 +48,28 @@ public SequenceFileReader(FileSystem fs, Path filePath, Map<String, Object> conf
42
48
this .key = (Writable ) ReflectionUtils .newInstance (reader .getKeyClass (), fs .getConf ());
43
49
this .value = (Writable ) ReflectionUtils .newInstance (reader .getValueClass (), fs .getConf ());
44
50
this .schema = SchemaBuilder .struct ()
45
- .field (FIELD_KEY , getSchema (key )).field (FIELD_VALUE , getSchema (value )).build ();
51
+ .field (keyFieldName , getSchema (this .key ))
52
+ .field (valueFieldName , getSchema (this .value ))
53
+ .build ();
46
54
this .offset = new SeqOffset (0 );
47
55
this .recordIndex = this .hasNextIndex = -1 ;
48
56
this .hasNext = false ;
49
57
}
50
58
51
59
@ Override
52
60
protected void configure (Map <String , Object > config ) {
61
+ if (config .get (FILE_READER_SEQUENCE_FIELD_NAME_KEY ) == null ||
62
+ config .get (FILE_READER_SEQUENCE_FIELD_NAME_KEY ).toString ().equals ("" )) {
63
+ this .keyFieldName = FIELD_NAME_KEY_DEFAULT ;
64
+ } else {
65
+ this .keyFieldName = config .get (FILE_READER_SEQUENCE_FIELD_NAME_KEY ).toString ();
66
+ }
67
+ if (config .get (FILE_READER_SEQUENCE_FIELD_NAME_VALUE ) == null ||
68
+ config .get (FILE_READER_SEQUENCE_FIELD_NAME_VALUE ).toString ().equals ("" )) {
69
+ this .valueFieldName = FIELD_NAME_VALUE_DEFAULT ;
70
+ } else {
71
+ this .valueFieldName = config .get (FILE_READER_SEQUENCE_FIELD_NAME_VALUE ).toString ();
72
+ }
53
73
}
54
74
55
75
private Schema getSchema (Writable writable ) {
@@ -95,7 +115,7 @@ protected SequenceRecord<Writable, Writable> nextRecord() {
95
115
throw new NoSuchElementException ("There are no more records in file: " + getFilePath ());
96
116
}
97
117
recordIndex ++;
98
- return new SequenceRecord <Writable , Writable >(schema , key , value );
118
+ return new SequenceRecord <Writable , Writable >(schema , keyFieldName , key , valueFieldName , value );
99
119
}
100
120
101
121
@ Override
@@ -149,8 +169,8 @@ static class SeqToStruct implements ReaderAdapter<SequenceRecord<Writable, Writa
149
169
@ Override
150
170
public Struct apply (SequenceRecord <Writable , Writable > record ) {
151
171
return new Struct (record .schema )
152
- .put (FIELD_KEY , toSchemaValue (record .key ))
153
- .put (FIELD_VALUE , toSchemaValue (record .value ));
172
+ .put (record . keyFieldName , toSchemaValue (record .key ))
173
+ .put (record . valueFieldName , toSchemaValue (record .value ));
154
174
}
155
175
156
176
private Object toSchemaValue (Writable writable ) {
@@ -176,13 +196,17 @@ private Object toSchemaValue(Writable writable) {
176
196
}
177
197
178
198
static class SequenceRecord <T , U > {
179
- final Schema schema ;
180
- final T key ;
181
- final U value ;
199
+ private final Schema schema ;
200
+ private final String keyFieldName ;
201
+ private final T key ;
202
+ private final String valueFieldName ;
203
+ private final U value ;
182
204
183
- public SequenceRecord (Schema schema , T key , U value ) {
205
+ public SequenceRecord (Schema schema , String keyFieldName , T key , String valueFieldName , U value ) {
184
206
this .schema = schema ;
207
+ this .keyFieldName = keyFieldName ;
185
208
this .key = key ;
209
+ this .valueFieldName = valueFieldName ;
186
210
this .value = value ;
187
211
}
188
212
0 commit comments