embulk-input-hdfs 0.2.1 → 0.3.0

Sign up to get free protection for your applications and to get access to all the features.
checksums.yaml CHANGED
@@ -1,7 +1,7 @@
1
1
  ---
2
2
  SHA1:
3
- metadata.gz: e666bbbcb18941dce84889c2ee7fb85d65edbaf4
4
- data.tar.gz: 7422b508396787d70e6cea3fc534739c2c20c825
3
+ metadata.gz: f0ecc7937c596a6725e8c9f95804132af16d94dc
4
+ data.tar.gz: 0312623c3baf28b75a42839f565a6f0671afc2b8
5
5
  SHA512:
6
- metadata.gz: c305947dbd3f6bded0a23fbc06efd4d44e6d48cdb4b97c8b0e3861cd4b2a9800f6d8c93cf5280ccb235ca88346e727bb5fb549ae3c7bb2e12a13205e20765085
7
- data.tar.gz: 8f33bb06731a3c5a25dd723bef83616992ce5fc8b8d5e1a60d8a1da56421a42b49ae3397feb24134a093bf291af87ddbd208fa866c86fdd997d824a6077434a4
6
+ metadata.gz: 6a4315e6f9c1dda4f752bcbe930373032b63c019c65bd9356ac385bcce7543fbb8e9279bb5204221f7dd2bef515259c2c19c0e69b1c8d148097567fa599b2c26
7
+ data.tar.gz: 589a462546b0b1cd376757f61179b857aba5d056b96c6864cc991617929ea6dc5b9255088c79766d54b8784fdc922b4f5192a90a913df0d424bb6fc5e64ea9a8
data/CHENGELOG.md CHANGED
@@ -1,3 +1,8 @@
1
+ 0.3.0 (2016-09-21)
2
+ ==================
3
+ - [Incompatible Change] Not partitoning if files are compressed
4
+ - https://github.com/civitaspo/embulk-input-hdfs/pull/27
5
+
1
6
  0.2.1 (2016-02-25)
2
7
  ==================
3
8
  - [Fix] does not work
data/README.md CHANGED
@@ -77,23 +77,26 @@ int partitionSizeByOneTask = totalFileLength / approximateNumPartitions;
77
77
  /*
78
78
  ...
79
79
  */
80
-
81
- long numPartitions;
82
- if (task.getPartition()) {
83
- if (file.canDecompress()) {
84
- numPartitions = ((fileLength - 1) / partitionSizeByOneTask) + 1;
85
- }
86
- else if (file.getCodec() != null) { // if not null, the file is compressed.
87
- numPartitions = 1;
88
- }
89
- else {
90
- numPartitions = ((fileLength - 1) / partitionSizeByOneTask) + 1;
91
- }
92
- }
93
- else {
94
- numPartitions = 1;
95
- }
96
-
80
+ long numPartitions = 1; // default is no partition.
81
+ if (isPartitionable(task, conf, status)) { // partition: true and (decompression: false or CompressionCodec is null)
82
+ numPartitions = ((status.getLen() - 1) / partitionSizeByOneTask) + 1;
83
+ }
84
+
85
+ for (long i = 0; i < numPartitions; i++) {
86
+ long start = status.getLen() * i / numPartitions;
87
+ long end = status.getLen() * (i + 1) / numPartitions;
88
+ if (start < end) {
89
+ TargetFileInfo targetFileInfo = new TargetFileInfo.Builder()
90
+ .pathString(status.getPath().toString())
91
+ .start(start)
92
+ .end(end)
93
+ .isDecompressible(isDecompressible(task, conf, status))
94
+ .isPartitionable(isPartitionable(task, conf, status))
95
+ .numHeaderLines(task.getSkipHeaderLines())
96
+ .build();
97
+ builder.add(targetFileInfo);
98
+ }
99
+ }
97
100
  /*
98
101
  ...
99
102
  */
data/build.gradle CHANGED
@@ -15,7 +15,7 @@ configurations {
15
15
  provided
16
16
  }
17
17
 
18
- version = "0.2.1"
18
+ version = "0.3.0"
19
19
 
20
20
  sourceCompatibility = 1.7
21
21
  targetCompatibility = 1.7
@@ -0,0 +1,60 @@
1
+ package org.embulk.input.hdfs;
2
+
3
+ import org.apache.hadoop.conf.Configuration;
4
+ import org.embulk.config.Config;
5
+ import org.embulk.config.ConfigDefault;
6
+ import org.embulk.config.ConfigException;
7
+ import org.embulk.spi.Exec;
8
+ import org.slf4j.Logger;
9
+
10
+ import java.io.File;
11
+ import java.net.MalformedURLException;
12
+ import java.util.List;
13
+ import java.util.Map;
14
+
15
+ public class ConfigurationFactory
16
+ {
17
+ public static final Logger logger = Exec.getLogger(ConfigurationFactory.class);
18
+
19
+ interface Task
20
+ {
21
+ @Config("config_files")
22
+ @ConfigDefault("[]")
23
+ List<String> getConfigFiles();
24
+
25
+ @Config("config")
26
+ @ConfigDefault("{}")
27
+ Map<String, String> getConfig();
28
+ }
29
+
30
+ private ConfigurationFactory()
31
+ {
32
+ }
33
+
34
+ public static Configuration create(Task task)
35
+ {
36
+ Configuration c = new Configuration();
37
+ for (String f : task.getConfigFiles()) {
38
+ try {
39
+ logger.debug("embulk-input-hdfs: load a config file: {}", f);
40
+ c.addResource(new File(f).toURI().toURL());
41
+ }
42
+ catch (MalformedURLException e) {
43
+ throw new ConfigException(e);
44
+ }
45
+ }
46
+
47
+ for (Map.Entry<String, String> entry : task.getConfig().entrySet()) {
48
+ logger.debug("embulk-input-hdfs: load a config: {}:{}", entry.getKey(), entry.getValue());
49
+ c.set(entry.getKey(), entry.getValue());
50
+ }
51
+
52
+ // For logging
53
+ for (Map.Entry<String, String> entry : c) {
54
+ logger.trace("embulk-input-hdfs: {}: {}", entry.getKey(), entry.getValue());
55
+ }
56
+ logger.trace("embulk-input-hdfs: Resource Files: {}", c);
57
+
58
+ return c;
59
+ }
60
+ }
@@ -1,17 +1,12 @@
1
1
  package org.embulk.input.hdfs;
2
2
 
3
- import com.google.common.annotations.VisibleForTesting;
4
- import com.google.common.base.Optional;
5
3
  import com.google.common.base.Throwables;
6
4
  import com.google.common.collect.Lists;
7
- import com.google.common.collect.Maps;
8
5
  import org.apache.hadoop.conf.Configuration;
9
6
  import org.apache.hadoop.fs.FileStatus;
10
7
  import org.apache.hadoop.fs.FileSystem;
11
8
  import org.apache.hadoop.fs.Path;
12
- import org.apache.hadoop.fs.PathIOException;
13
9
  import org.apache.hadoop.fs.PathNotFoundException;
14
- import org.apache.hadoop.io.compress.CompressionCodec;
15
10
  import org.apache.hadoop.io.compress.CompressionCodecFactory;
16
11
  import org.embulk.config.Config;
17
12
  import org.embulk.config.ConfigDefault;
@@ -22,43 +17,30 @@ import org.embulk.config.Task;
22
17
  import org.embulk.config.TaskReport;
23
18
  import org.embulk.config.TaskSource;
24
19
  import org.embulk.spi.BufferAllocator;
20
+ import org.embulk.spi.DataException;
25
21
  import org.embulk.spi.Exec;
26
22
  import org.embulk.spi.FileInputPlugin;
27
23
  import org.embulk.spi.TransactionalFileInput;
28
24
  import org.embulk.spi.util.InputStreamFileInput;
29
- import org.jruby.embed.ScriptingContainer;
30
25
  import org.slf4j.Logger;
31
26
 
32
27
  import java.io.IOException;
33
28
  import java.io.InputStream;
34
29
  import java.util.Iterator;
35
30
  import java.util.List;
36
- import java.util.Map;
37
31
 
38
32
  public class HdfsFileInputPlugin
39
33
  implements FileInputPlugin
40
34
  {
41
35
  public interface PluginTask
42
- extends Task, PartialFileList.Task
36
+ extends Task, TargetFileInfoList.Task, ConfigurationFactory.Task, Strftime.Task
43
37
  {
44
- @Config("config_files")
45
- @ConfigDefault("[]")
46
- List<String> getConfigFiles();
47
-
48
- @Config("config")
49
- @ConfigDefault("{}")
50
- Map<String, String> getConfig();
51
-
52
38
  @Config("path")
53
39
  String getPath();
54
40
 
55
- @Config("rewind_seconds")
56
- @ConfigDefault("0")
57
- int getRewindSeconds();
58
-
59
41
  @Config("partition")
60
42
  @ConfigDefault("true")
61
- boolean getPartition();
43
+ boolean getWillPartition();
62
44
 
63
45
  @Config("num_partitions") // this parameter is the approximate value.
64
46
  @ConfigDefault("-1") // Default: Runtime.getRuntime().availableProcessors()
@@ -70,253 +52,175 @@ public class HdfsFileInputPlugin
70
52
 
71
53
  @Config("decompression") // if true, decompress files by using compression codec
72
54
  @ConfigDefault("false") // when getting FileInputStream.
73
- boolean getDecompression();
55
+ boolean getWillDecompress();
74
56
 
75
- PartialFileList getPartialFileList();
76
- void setPartialFileList(PartialFileList partialFileList);
77
-
78
- @ConfigInject
79
- ScriptingContainer getJRuby();
80
-
81
- @ConfigInject
82
- BufferAllocator getBufferAllocator();
57
+ TargetFileInfoList getTargetFileInfoList();
58
+ void setTargetFileInfoList(TargetFileInfoList targetFileInfoList);
83
59
  }
84
60
 
85
61
  private static final Logger logger = Exec.getLogger(HdfsFileInputPlugin.class);
86
- private Optional<Configuration> configurationContainer = Optional.absent();
87
62
 
88
63
  @Override
89
64
  public ConfigDiff transaction(ConfigSource config, FileInputPlugin.Control control)
90
65
  {
91
66
  PluginTask task = config.loadConfig(PluginTask.class);
92
- Configuration configuration = getConfiguration(task);
67
+
68
+ if (task.getWillPartition() && task.getWillDecompress()) {
69
+ logger.info("embulk-input-hdfs: Please be sure that the target files cannot be partitioned if they are compressed.");
70
+ }
71
+
72
+ Configuration conf = ConfigurationFactory.create(task);
93
73
 
94
74
  // listing Files
95
75
  try {
96
- FileSystem fs = getFS(configuration);
76
+ FileSystem fs = FileSystem.get(conf);
97
77
 
98
- String pathString = strftime(task.getJRuby(), task.getPath(), task.getRewindSeconds());
78
+ String pathString = new Strftime(task).format(task.getPath());
99
79
  Path rootPath = new Path(pathString);
100
80
 
101
- List<Path> originalFileList = buildOriginalFileList(fs, rootPath);
81
+ List<FileStatus> statusList = listFileStatuses(fs, rootPath);
102
82
 
103
- if (originalFileList.isEmpty()) {
83
+ if (statusList.isEmpty()) {
104
84
  throw new PathNotFoundException(pathString);
105
85
  }
106
86
 
107
- logger.debug("embulk-input-hdfs: Loading target files: {}", originalFileList);
108
- PartialFileList list = buildPartialFileList(task, originalFileList);
109
- task.setPartialFileList(list);
87
+ for (FileStatus status : statusList) {
88
+ logger.debug("embulk-input-hdfs: Loading paths: {}, length: {}", status.getPath(), status.getLen());
89
+ }
90
+
91
+ TargetFileInfoList list = buildTargetFileInfoList(task, statusList);
92
+ task.setTargetFileInfoList(list);
110
93
  }
111
94
  catch (IOException e) {
112
95
  logger.error(e.getMessage());
113
- throw new RuntimeException(e);
96
+ throw Throwables.propagate(e);
114
97
  }
115
98
 
116
99
  // number of processors is same with number of targets
117
- int taskCount = task.getPartialFileList().getTaskCount();
100
+ int taskCount = task.getTargetFileInfoList().getTaskCount();
118
101
  logger.info("embulk-input-hdfs: task size: {}", taskCount);
119
102
 
120
103
  return resume(task.dump(), taskCount, control);
121
104
  }
122
105
 
123
- private Configuration getConfiguration(PluginTask task)
106
+ private List<FileStatus> listFileStatuses(FileSystem fs, Path rootPath)
107
+ throws IOException
124
108
  {
125
- if (configurationContainer.isPresent()) {
126
- return configurationContainer.get();
127
- }
128
-
129
- ConfigurationBuilder builder = new ConfigurationBuilder();
130
- builder.addConfigFiles(task.getConfigFiles());
131
- builder.addConfigMap(task.getConfig());
132
- configurationContainer = Optional.of(builder.build());
133
- return configurationContainer.get();
134
- }
109
+ List<FileStatus> statusList = Lists.newArrayList();
135
110
 
136
- private FileSystem getFS(Configuration configuration)
137
- {
138
- try {
139
- return FileSystem.get(configuration);
140
- }
141
- catch (IOException e) {
142
- throw Throwables.propagate(e);
143
- }
144
- }
145
-
146
- @VisibleForTesting
147
- String strftime(final ScriptingContainer jruby, final String format, final int rewindSeconds)
148
- {
149
- String script = String.format("(Time.now - %d).strftime('%s')", rewindSeconds, format);
150
- return jruby.runScriptlet(script).toString();
151
- }
152
-
153
- private List<Path> buildOriginalFileList(FileSystem fs, Path rootPath)
154
- {
155
- List<Path> fileList = Lists.newArrayList();
156
-
157
- final FileStatus[] entries;
158
- try {
159
- entries = fs.globStatus(rootPath);
160
- }
161
- catch (IOException e) {
162
- throw Throwables.propagate(e);
163
- }
111
+ FileStatus[] entries = fs.globStatus(rootPath);
164
112
  // `globStatus` does not throw PathNotFoundException.
165
113
  // return null instead.
166
114
  // see: https://github.com/apache/hadoop/blob/branch-2.7.0/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/Globber.java#L286
167
115
  if (entries == null) {
168
- return fileList;
116
+ return statusList;
169
117
  }
170
118
 
171
119
  for (FileStatus entry : entries) {
172
120
  if (entry.isDirectory()) {
173
- List<Path> subEntries = listRecursive(fs, entry);
174
- fileList.addAll(subEntries);
121
+ // TODO: use fs.listFiles(entry.getPath(), true); ?
122
+ List<FileStatus> subEntries = listRecursive(fs, entry);
123
+ statusList.addAll(subEntries);
175
124
  }
176
125
  else {
177
- fileList.add(entry.getPath());
126
+ statusList.add(entry);
178
127
  }
179
128
  }
180
129
 
181
- return fileList;
130
+ return statusList;
182
131
  }
183
132
 
184
- private List<Path> listRecursive(final FileSystem fs, FileStatus status)
133
+ private List<FileStatus> listRecursive(FileSystem fs, FileStatus status)
134
+ throws IOException
185
135
  {
186
- List<Path> fileList = Lists.newArrayList();
136
+ List<FileStatus> statusList = Lists.newArrayList();
187
137
  if (status.isDirectory()) {
188
- FileStatus[] entries;
189
- try {
190
- entries = fs.listStatus(status.getPath());
191
- }
192
- catch (IOException e) {
193
- throw Throwables.propagate(e);
194
- }
195
-
138
+ FileStatus[] entries = fs.listStatus(status.getPath());
196
139
  for (FileStatus entry : entries) {
197
- fileList.addAll(listRecursive(fs, entry));
140
+ statusList.addAll(listRecursive(fs, entry));
198
141
  }
199
142
  }
200
143
  else {
201
- fileList.add(status.getPath());
144
+ statusList.add(status);
202
145
  }
203
- return fileList;
146
+ return statusList;
204
147
  }
205
148
 
206
- private PartialFileList buildPartialFileList(PluginTask task, List<Path> pathList)
149
+ private TargetFileInfoList buildTargetFileInfoList(PluginTask task, List<FileStatus> statusList)
150
+ throws IOException, DataException
207
151
  {
208
- Configuration configuration = getConfiguration(task);
209
- FileSystem fs = getFS(configuration);
210
- boolean shouldPartition = task.getPartition();
211
- boolean shouldDecompress = task.getDecompression();
212
-
213
- Map<Path, Long> pathLengthMap = Maps.newHashMap();
214
- long totalFileLength = 0;
215
- for (Path path : pathList) {
216
- long fileLength = getHdfsFileLength(fs, path, shouldDecompress);
217
-
218
- if (fileLength <= 0) {
219
- logger.info("Skip the 0 byte target file: {}", path);
220
- continue;
221
- }
222
-
223
- pathLengthMap.put(path, fileLength);
224
- totalFileLength += fileLength;
225
- }
152
+ long totalFileLength = calcTotalFilesLength(statusList);
226
153
  if (totalFileLength <= 0) {
227
- throw Throwables.propagate(new PathIOException(task.getPath(), "All files are empty"));
228
- }
229
-
230
- PartialFileList.Builder builder = new PartialFileList.Builder(task);
231
-
232
- // TODO: optimum allocation of resources
233
- final long approximateNumPartitions;
234
- if (task.getApproximateNumPartitions() <= 0) {
235
- approximateNumPartitions = Runtime.getRuntime().availableProcessors();
236
- }
237
- else {
238
- approximateNumPartitions = task.getApproximateNumPartitions();
154
+ // TODO: skip this error because other file input plugins have no errors if files are empty.
155
+ throw new DataException("embulk-input-hdfs: All files are empty: " + task.getPath());
239
156
  }
240
157
 
241
- long partitionSizeByOneTask = totalFileLength / approximateNumPartitions;
242
- if (partitionSizeByOneTask <= 0) {
243
- partitionSizeByOneTask = 1;
244
- }
245
-
246
- for (Map.Entry<Path, Long> entry : pathLengthMap.entrySet()) {
247
- Path path = entry.getKey();
248
- long fileLength = entry.getValue();
158
+ long partitionSizeByOneTask = calcApproximatePartitionSizeByOneTask(task, totalFileLength);
249
159
 
250
- long numPartitions;
251
- if (shouldPartition) {
252
- if (shouldDecompress && getHdfsFileCompressionCodec(fs, path) != null) {
253
- numPartitions = ((fileLength - 1) / partitionSizeByOneTask) + 1;
254
- }
255
- else if (getHdfsFileCompressionCodec(fs, path) != null) { // if not null, the file is compressed.
256
- numPartitions = 1;
257
- }
258
- else {
259
- numPartitions = ((fileLength - 1) / partitionSizeByOneTask) + 1;
260
- }
160
+ Configuration conf = ConfigurationFactory.create(task);
161
+ TargetFileInfoList.Builder builder = TargetFileInfoList.builder(task);
162
+ for (FileStatus status : statusList) {
163
+ if (status.getLen() <= 0) {
164
+ logger.info("embulk-input-hdfs: Skip the 0 byte target file: {}", status.getPath());
165
+ continue;
261
166
  }
262
- else {
263
- numPartitions = 1;
167
+
168
+ long numPartitions = 1; // default is no partition.
169
+ if (isPartitionable(task, conf, status)) {
170
+ numPartitions = ((status.getLen() - 1) / partitionSizeByOneTask) + 1;
264
171
  }
265
172
 
266
173
  for (long i = 0; i < numPartitions; i++) {
267
- long start = fileLength * i / numPartitions;
268
- long end = fileLength * (i + 1) / numPartitions;
174
+ long start = status.getLen() * i / numPartitions;
175
+ long end = status.getLen() * (i + 1) / numPartitions;
269
176
  if (start < end) {
270
- logger.debug("PartialFile: path {}, start: {}, end: {}", path, start, end);
271
- builder.add(path.toString(), start, end, shouldDecompress && getHdfsFileCompressionCodec(fs, path) != null);
177
+ TargetFileInfo targetFileInfo = new TargetFileInfo.Builder()
178
+ .pathString(status.getPath().toString())
179
+ .start(start)
180
+ .end(end)
181
+ .isDecompressible(isDecompressible(task, conf, status))
182
+ .isPartitionable(isPartitionable(task, conf, status))
183
+ .numHeaderLines(task.getSkipHeaderLines())
184
+ .build();
185
+ builder.add(targetFileInfo);
272
186
  }
273
187
  }
274
188
  }
275
-
276
189
  return builder.build();
277
190
  }
278
191
 
279
- private Long getHdfsFileLength(FileSystem fs, Path path, boolean shouldDecompression)
192
+ private boolean isDecompressible(PluginTask task, Configuration conf, FileStatus status)
280
193
  {
281
- CompressionCodec codec = getHdfsFileCompressionCodec(fs, path);
282
- if (codec == null) {
283
- try {
284
- return fs.getFileStatus(path).getLen();
285
- }
286
- catch (IOException e) {
287
- throw Throwables.propagate(e);
288
- }
289
- }
290
- else if (!shouldDecompression) {
291
- try {
292
- return fs.getFileStatus(path).getLen();
293
- }
294
- catch (IOException e) {
295
- throw Throwables.propagate(e);
296
- }
297
- }
298
- else {
299
- long fileLength = 0;
300
- try (InputStream is = codec.createInputStream(fs.open(path))) {
301
- while (is.read() > 0) {
302
- fileLength++;
303
- }
304
- }
305
- catch (IOException e) {
306
- throw Throwables.propagate(e);
307
- }
308
- return fileLength;
309
- }
194
+ return task.getWillDecompress() && new CompressionCodecFactory(conf).getCodec(status.getPath()) != null;
310
195
  }
311
196
 
312
- private CompressionCodec getHdfsFileCompressionCodec(FileSystem fs, Path path)
197
+ private boolean isPartitionable(PluginTask task, Configuration conf, FileStatus status)
313
198
  {
314
- return getHdfsFileCompressionCodec(fs.getConf(), path);
199
+ return task.getWillPartition() && !isDecompressible(task, conf, status);
315
200
  }
316
201
 
317
- private CompressionCodec getHdfsFileCompressionCodec(Configuration configuration, Path path)
202
+ private long calcTotalFilesLength(List<FileStatus> statusList)
203
+ throws IOException
318
204
  {
319
- return new CompressionCodecFactory(configuration).getCodec(path);
205
+ long total = 0L;
206
+ for (FileStatus status : statusList) {
207
+ total += status.getLen();
208
+ }
209
+ return total;
210
+ }
211
+
212
+ private long calcApproximatePartitionSizeByOneTask(PluginTask task, long totalFilesLength)
213
+ {
214
+ long numPartitions = task.getApproximateNumPartitions();
215
+ if (numPartitions <= 0) {
216
+ numPartitions = Runtime.getRuntime().availableProcessors();
217
+ }
218
+ // TODO: optimum allocation of resources
219
+ long partitionSizeByOneTask = totalFilesLength / numPartitions;
220
+ if (partitionSizeByOneTask <= 0) {
221
+ partitionSizeByOneTask = 1;
222
+ }
223
+ return partitionSizeByOneTask;
320
224
  }
321
225
 
322
226
  @Override
@@ -340,17 +244,22 @@ public class HdfsFileInputPlugin
340
244
  public TransactionalFileInput open(TaskSource taskSource, int taskIndex)
341
245
  {
342
246
  final PluginTask task = taskSource.loadTask(PluginTask.class);
343
- return new HdfsFileInput(task, taskIndex);
247
+ try {
248
+ return new HdfsFileInput(task, taskIndex);
249
+ }
250
+ catch (IOException e) {
251
+ throw Throwables.propagate(e);
252
+ }
344
253
  }
345
254
 
346
255
  public class HdfsFileInput
347
256
  extends InputStreamFileInput
348
257
  implements TransactionalFileInput
349
258
  {
350
-
351
259
  public HdfsFileInput(PluginTask task, int taskIndex)
260
+ throws IOException
352
261
  {
353
- super(task.getBufferAllocator(), new SingleFileProvider(task, taskIndex));
262
+ super(Exec.getBufferAllocator(), new SingleFileProvider(task, taskIndex));
354
263
  }
355
264
 
356
265
  @Override
@@ -369,15 +278,14 @@ public class HdfsFileInputPlugin
369
278
  private class SingleFileProvider
370
279
  implements InputStreamFileInput.Provider
371
280
  {
372
- private final FileSystem fs;
373
- private final int numHeaderLines;
374
- private final Iterator<PartialFile> iterator;
281
+ private final TargetFileInputStreamFactory factory;
282
+ private final Iterator<TargetFileInfo> iterator;
375
283
 
376
284
  public SingleFileProvider(PluginTask task, int taskIndex)
285
+ throws IOException
377
286
  {
378
- this.fs = getFS(getConfiguration(task));
379
- this.numHeaderLines = task.getSkipHeaderLines();
380
- this.iterator = task.getPartialFileList().get(taskIndex).iterator();
287
+ this.factory = new TargetFileInputStreamFactory(FileSystem.get(ConfigurationFactory.create(task)));
288
+ this.iterator = task.getTargetFileInfoList().get(taskIndex).iterator();
381
289
  }
382
290
 
383
291
  @Override
@@ -386,8 +294,7 @@ public class HdfsFileInputPlugin
386
294
  if (!iterator.hasNext()) {
387
295
  return null;
388
296
  }
389
- PartialFileInputStreamBuilder builder = new PartialFileInputStreamBuilder(fs, iterator.next()).withHeaders(numHeaderLines);
390
- return builder.build();
297
+ return factory.create(iterator.next());
391
298
  }
392
299
 
393
300
  @Override