-
Notifications
You must be signed in to change notification settings - Fork 4.1k
/
HdfsSpout.java
836 lines (744 loc) · 31.9 KB
/
HdfsSpout.java
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
/**
* Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership. The ASF licenses this file to you under the Apache License, Version
* 2.0 (the "License"); you may not use this file except in compliance with the License. You may obtain a copy of the License at
* <p/>
* http://www.apache.org/licenses/LICENSE-2.0
* <p/>
* Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the specific language governing permissions
* and limitations under the License.
*/
package org.apache.storm.hdfs.spout;
import java.io.IOException;
import java.lang.reflect.Constructor;
import java.net.URI;
import java.util.Collection;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Timer;
import java.util.TimerTask;
import java.util.concurrent.LinkedBlockingQueue;
import java.util.concurrent.atomic.AtomicBoolean;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.storm.Config;
import org.apache.storm.hdfs.common.HdfsUtils;
import org.apache.storm.hdfs.security.HdfsSecurityUtil;
import org.apache.storm.spout.SpoutOutputCollector;
import org.apache.storm.task.TopologyContext;
import org.apache.storm.topology.OutputFieldsDeclarer;
import org.apache.storm.topology.base.BaseRichSpout;
import org.apache.storm.tuple.Fields;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
public class HdfsSpout extends BaseRichSpout {
// other members
private static final Logger LOG = LoggerFactory.getLogger(HdfsSpout.class);
private final AtomicBoolean commitTimeElapsed = new AtomicBoolean(false);
HashMap<MessageId, List<Object>> inflight = new HashMap<>();
LinkedBlockingQueue<HdfsUtils.Pair<MessageId, List<Object>>> retryList = new LinkedBlockingQueue<>();
HdfsUtils.Pair<Path, FileLock.LogEntry> lastExpiredLock = null;
// user configurable
private String hdfsUri; // required
private String readerType; // required
private Fields outputFields; // required
private String sourceDir; // required
private Path sourceDirPath; // required
private String archiveDir; // required
private Path archiveDirPath; // required
private String badFilesDir; // required
private Path badFilesDirPath; // required
private String lockDir;
private Path lockDirPath;
private int commitFrequencyCount = Configs.DEFAULT_COMMIT_FREQ_COUNT;
private int commitFrequencySec = Configs.DEFAULT_COMMIT_FREQ_SEC;
private int maxOutstanding = Configs.DEFAULT_MAX_OUTSTANDING;
private int lockTimeoutSec = Configs.DEFAULT_LOCK_TIMEOUT;
private boolean clocksInSync = true;
private String inprogressSuffix = ".inprogress"; // not configurable to prevent change between topology restarts
private String ignoreSuffix = ".ignore";
private String outputStreamName = null;
private ProgressTracker tracker = null;
private FileSystem hdfs;
private FileReader reader;
private SpoutOutputCollector collector;
private Configuration hdfsConfig;
private Map<String, Object> conf = null;
private FileLock lock;
private String spoutId = null;
private long lastExpiredLockTime = 0;
private long tupleCounter = 0;
private boolean ackEnabled = false;
private int acksSinceLastCommit = 0;
private Timer commitTimer;
private boolean fileReadCompletely = true;
private String configKey = Configs.DEFAULT_HDFS_CONFIG_KEY; // key for hdfs Kerberos configs
public HdfsSpout() {
}
private static String getFileProgress(FileReader reader) {
return reader.getFilePath() + " " + reader.getFileOffset();
}
private static void releaseLockAndLog(FileLock fileLock, String spoutId) {
try {
if (fileLock != null) {
fileLock.release();
LOG.debug("Spout {} released FileLock. SpoutId = {}", fileLock.getLockFile(), spoutId);
}
} catch (IOException e) {
LOG.error("Unable to delete lock file : " + fileLock.getLockFile() + " SpoutId =" + spoutId, e);
}
}
private static void validateOrMakeDir(FileSystem fs, Path dir, String dirDescription) {
try {
if (fs.exists(dir)) {
if (!fs.isDirectory(dir)) {
LOG.error(dirDescription + " directory is a file, not a dir. " + dir);
throw new RuntimeException(dirDescription + " directory is a file, not a dir. " + dir);
}
} else if (!fs.mkdirs(dir)) {
LOG.error("Unable to create " + dirDescription + " directory " + dir);
throw new RuntimeException("Unable to create " + dirDescription + " directory " + dir);
}
} catch (IOException e) {
LOG.error("Unable to create " + dirDescription + " directory " + dir, e);
throw new RuntimeException("Unable to create " + dirDescription + " directory " + dir, e);
}
}
static void checkValidReader(String readerType) {
if (readerType.equalsIgnoreCase(Configs.TEXT) || readerType.equalsIgnoreCase(Configs.SEQ)) {
return;
}
try {
Class<?> classType = Class.forName(readerType);
classType.getConstructor(FileSystem.class, Path.class, Map.class);
if (!FileReader.class.isAssignableFrom(classType)) {
LOG.error(readerType + " not a FileReader");
throw new IllegalArgumentException(readerType + " not a FileReader.");
}
return;
} catch (ClassNotFoundException e) {
LOG.error(readerType + " not found in classpath.", e);
throw new IllegalArgumentException(readerType + " not found in classpath.", e);
} catch (NoSuchMethodException e) {
LOG.error(readerType + " is missing the expected constructor for Readers.", e);
throw new IllegalArgumentException(readerType + " is missing the expected constuctor for Readers.");
}
}
public HdfsSpout setHdfsUri(String hdfsUri) {
this.hdfsUri = hdfsUri;
return this;
}
public HdfsSpout setReaderType(String readerType) {
this.readerType = readerType;
return this;
}
public HdfsSpout setSourceDir(String sourceDir) {
this.sourceDir = sourceDir;
return this;
}
public HdfsSpout setArchiveDir(String archiveDir) {
this.archiveDir = archiveDir;
return this;
}
public HdfsSpout setBadFilesDir(String badFilesDir) {
this.badFilesDir = badFilesDir;
return this;
}
public HdfsSpout setLockDir(String lockDir) {
this.lockDir = lockDir;
return this;
}
public HdfsSpout setCommitFrequencyCount(int commitFrequencyCount) {
this.commitFrequencyCount = commitFrequencyCount;
return this;
}
public HdfsSpout setCommitFrequencySec(int commitFrequencySec) {
this.commitFrequencySec = commitFrequencySec;
return this;
}
public HdfsSpout setMaxOutstanding(int maxOutstanding) {
this.maxOutstanding = maxOutstanding;
return this;
}
public HdfsSpout setLockTimeoutSec(int lockTimeoutSec) {
this.lockTimeoutSec = lockTimeoutSec;
return this;
}
public HdfsSpout setClocksInSync(boolean clocksInSync) {
this.clocksInSync = clocksInSync;
return this;
}
public HdfsSpout setIgnoreSuffix(String ignoreSuffix) {
this.ignoreSuffix = ignoreSuffix;
return this;
}
/**
* Output field names. Number of fields depends upon the reader type
*/
public HdfsSpout withOutputFields(String... fields) {
outputFields = new Fields(fields);
return this;
}
/**
* set key name under which HDFS options are placed. (similar to HDFS bolt). default key name is 'hdfs.config'
*/
public HdfsSpout withConfigKey(String configKey) {
this.configKey = configKey;
return this;
}
/**
* Set output stream name.
*/
public HdfsSpout withOutputStream(String streamName) {
this.outputStreamName = streamName;
return this;
}
public Path getLockDirPath() {
return lockDirPath;
}
public SpoutOutputCollector getCollector() {
return collector;
}
@Override
public void nextTuple() {
LOG.trace("Next Tuple {}", spoutId);
// 1) First re-emit any previously failed tuples (from retryList)
if (!retryList.isEmpty()) {
LOG.debug("Sending tuple from retry list");
HdfsUtils.Pair<MessageId, List<Object>> pair = retryList.remove();
emitData(pair.getValue(), pair.getKey());
return;
}
if (ackEnabled && tracker.size() >= maxOutstanding) {
LOG.warn("Waiting for more ACKs before generating new tuples. "
+ "Progress tracker size has reached limit {}, SpoutID {}",
maxOutstanding, spoutId);
// Don't emit anything .. allow configured spout wait strategy to kick in
return;
}
// 2) If no failed tuples to be retried, then send tuples from hdfs
while (true) {
try {
// 3) Select a new file if one is not open already
boolean newReader = false;
if (reader == null) {
reader = pickNextFile();
if (reader == null) {
LOG.debug("Currently no new files to process under : " + sourceDirPath);
return;
} else {
fileReadCompletely = false;
newReader = true;
}
}
if (fileReadCompletely) { // wait for more ACKs before proceeding
return;
}
// 4) Read record from file, emit to collector and record progress
List<Object> tuple = reader.next();
if (tuple != null) {
fileReadCompletely = false;
++tupleCounter;
MessageId msgId = new MessageId(tupleCounter, reader.getFilePath(), reader.getFileOffset());
emitData(tuple, msgId);
if (!ackEnabled) {
++acksSinceLastCommit; // assume message is immediately ACKed in non-ack mode
commitProgress(reader.getFileOffset());
} else {
commitProgress(tracker.getCommitPosition());
}
return;
} else {
fileReadCompletely = true;
// if newReader is true and tuple is null then it is an empty reader
if (!ackEnabled || newReader) {
markFileAsDone(reader.getFilePath());
}
}
} catch (IOException e) {
LOG.error("I/O Error processing at file location " + getFileProgress(reader), e);
// don't emit anything .. allow configured spout wait strategy to kick in
return;
} catch (ParseException e) {
LOG.error("Parsing error when processing at file location " + getFileProgress(reader)
+ ". Skipping remainder of file.", e);
markFileAsBad(reader.getFilePath());
// Note: We don't return from this method on ParseException to avoid triggering the
// spout wait strategy (due to no emits). Instead we go back into the loop and
// generate a tuple from next file
}
} // while
}
// will commit progress into lock file if commit threshold is reached
private void commitProgress(FileOffset position) {
if (position == null) {
return;
}
if (lock != null && canCommitNow()) {
try {
String pos = position.toString();
lock.heartbeat(pos);
LOG.debug("{} Committed progress. {}", spoutId, pos);
acksSinceLastCommit = 0;
commitTimeElapsed.set(false);
setupCommitElapseTimer();
} catch (IOException e) {
LOG.error("Unable to commit progress Will retry later. Spout ID = " + spoutId, e);
}
}
}
private void setupCommitElapseTimer() {
if (commitFrequencySec <= 0) {
return;
}
TimerTask timerTask = new TimerTask() {
@Override
public void run() {
commitTimeElapsed.set(true);
}
};
commitTimer.schedule(timerTask, commitFrequencySec * 1000);
}
private void markFileAsDone(Path filePath) {
try {
Path newFile = renameCompletedFile(reader.getFilePath());
LOG.info("Completed processing {}. Spout Id = {}", newFile, spoutId);
} catch (IOException e) {
LOG.error("Unable to archive completed file" + filePath + " Spout ID " + spoutId, e);
}
closeReaderAndResetTrackers();
}
private void markFileAsBad(Path file) {
String fileName = file.toString();
String fileNameMinusSuffix = fileName.substring(0, fileName.indexOf(inprogressSuffix));
String originalName = new Path(fileNameMinusSuffix).getName();
Path newFile = new Path(badFilesDirPath + Path.SEPARATOR + originalName);
LOG.info("Moving bad file {} to {}. Processed it till offset {}. SpoutID= {}", originalName, newFile, tracker.getCommitPosition(),
spoutId);
try {
if (!hdfs.rename(file, newFile)) { // seems this can fail by returning false or throwing exception
throw new IOException("Move failed for bad file: " + file); // convert false ret value to exception
}
} catch (IOException e) {
LOG.warn("Error moving bad file: " + file + " to destination " + newFile + " SpoutId =" + spoutId, e);
}
closeReaderAndResetTrackers();
}
private void closeReaderAndResetTrackers() {
inflight.clear();
tracker.offsets.clear();
retryList.clear();
reader.close();
reader = null;
releaseLockAndLog(lock, spoutId);
lock = null;
}
protected void emitData(List<Object> tuple, MessageId id) {
LOG.trace("Emitting - {}", id);
if (outputStreamName == null) {
collector.emit(tuple, id);
} else {
collector.emit(outputStreamName, tuple, id);
}
inflight.put(id, tuple);
}
@SuppressWarnings("deprecation")
@Override
public void open(Map<String, Object> conf, TopologyContext context, SpoutOutputCollector collector) {
LOG.info("Opening HDFS Spout");
this.conf = conf;
this.commitTimer = new Timer(context.getThisTaskId() + "-commit-timer", true);
this.tracker = new ProgressTracker();
this.hdfsConfig = new Configuration();
this.collector = collector;
// Hdfs related settings
if (this.hdfsUri == null && conf.containsKey(Configs.HDFS_URI)) {
this.hdfsUri = conf.get(Configs.HDFS_URI).toString();
}
if (this.hdfsUri == null) {
throw new RuntimeException("HDFS Uri not set on spout");
}
try {
this.hdfs = FileSystem.get(URI.create(hdfsUri), hdfsConfig);
} catch (IOException e) {
LOG.error("Unable to instantiate file system", e);
throw new RuntimeException("Unable to instantiate file system", e);
}
if (conf.containsKey(configKey)) {
Map<String, Object> map = (Map<String, Object>) conf.get(configKey);
if (map != null) {
for (String keyName : map.keySet()) {
LOG.info("HDFS Config override : {} = {} ", keyName, String.valueOf(map.get(keyName)));
this.hdfsConfig.set(keyName, String.valueOf(map.get(keyName)));
}
try {
HdfsSecurityUtil.login(conf, hdfsConfig);
} catch (IOException e) {
LOG.error("HDFS Login failed ", e);
throw new RuntimeException(e);
}
} // if (map != null)
}
// Reader type config
if (readerType == null && conf.containsKey(Configs.READER_TYPE)) {
readerType = conf.get(Configs.READER_TYPE).toString();
}
checkValidReader(readerType);
// -- source dir config
if (sourceDir == null && conf.containsKey(Configs.SOURCE_DIR)) {
sourceDir = conf.get(Configs.SOURCE_DIR).toString();
}
if (sourceDir == null) {
LOG.error(Configs.SOURCE_DIR + " setting is required");
throw new RuntimeException(Configs.SOURCE_DIR + " setting is required");
}
this.sourceDirPath = new Path(sourceDir);
// -- archive dir config
if (archiveDir == null && conf.containsKey(Configs.ARCHIVE_DIR)) {
archiveDir = conf.get(Configs.ARCHIVE_DIR).toString();
}
if (archiveDir == null) {
LOG.error(Configs.ARCHIVE_DIR + " setting is required");
throw new RuntimeException(Configs.ARCHIVE_DIR + " setting is required");
}
this.archiveDirPath = new Path(archiveDir);
validateOrMakeDir(hdfs, archiveDirPath, "Archive");
// -- bad files dir config
if (badFilesDir == null && conf.containsKey(Configs.BAD_DIR)) {
badFilesDir = conf.get(Configs.BAD_DIR).toString();
}
if (badFilesDir == null) {
LOG.error(Configs.BAD_DIR + " setting is required");
throw new RuntimeException(Configs.BAD_DIR + " setting is required");
}
this.badFilesDirPath = new Path(badFilesDir);
validateOrMakeDir(hdfs, badFilesDirPath, "bad files");
// -- ignore file names config
if (conf.containsKey(Configs.IGNORE_SUFFIX)) {
this.ignoreSuffix = conf.get(Configs.IGNORE_SUFFIX).toString();
}
// -- lock dir config
if (lockDir == null && conf.containsKey(Configs.LOCK_DIR)) {
lockDir = conf.get(Configs.LOCK_DIR).toString();
}
if (lockDir == null) {
lockDir = getDefaultLockDir(sourceDirPath);
}
this.lockDirPath = new Path(lockDir);
validateOrMakeDir(hdfs, lockDirPath, "locks");
// -- lock timeout
if (conf.get(Configs.LOCK_TIMEOUT) != null) {
this.lockTimeoutSec = Integer.parseInt(conf.get(Configs.LOCK_TIMEOUT).toString());
}
// -- enable/disable ACKing
Object ackers = conf.get(Config.TOPOLOGY_ACKER_EXECUTORS);
if (ackers != null) {
int ackerCount = Integer.parseInt(ackers.toString());
this.ackEnabled = (ackerCount > 0);
LOG.debug("ACKer count = {}", ackerCount);
} else { // ackers==null when ackerCount not explicitly set on the topology
this.ackEnabled = true;
LOG.debug("ACK count not explicitly set on topology.");
}
LOG.info("ACK mode is {}", ackEnabled ? "enabled" : "disabled");
// -- commit frequency - count
if (conf.get(Configs.COMMIT_FREQ_COUNT) != null) {
commitFrequencyCount = Integer.parseInt(conf.get(Configs.COMMIT_FREQ_COUNT).toString());
}
// -- commit frequency - seconds
if (conf.get(Configs.COMMIT_FREQ_SEC) != null) {
commitFrequencySec = Integer.parseInt(conf.get(Configs.COMMIT_FREQ_SEC).toString());
if (commitFrequencySec <= 0) {
throw new RuntimeException(Configs.COMMIT_FREQ_SEC + " setting must be greater than 0");
}
}
// -- max outstanding tuples
if (conf.get(Configs.MAX_OUTSTANDING) != null) {
maxOutstanding = Integer.parseInt(conf.get(Configs.MAX_OUTSTANDING).toString());
}
// -- clocks in sync
if (conf.get(Configs.CLOCKS_INSYNC) != null) {
clocksInSync = Boolean.parseBoolean(conf.get(Configs.CLOCKS_INSYNC).toString());
}
// -- spout id
spoutId = context.getThisComponentId();
// setup timer for commit elapse time tracking
setupCommitElapseTimer();
}
@Override
public void close() {
this.commitTimer.cancel();
}
private String getDefaultLockDir(Path sourceDirPath) {
return sourceDirPath.toString() + Path.SEPARATOR + Configs.DEFAULT_LOCK_DIR;
}
@Override
public void ack(Object msgId) {
LOG.trace("Ack received for msg {} on spout {}", msgId, spoutId);
if (!ackEnabled) {
return;
}
MessageId id = (MessageId) msgId;
inflight.remove(id);
++acksSinceLastCommit;
tracker.recordAckedOffset(id.offset);
commitProgress(tracker.getCommitPosition());
if (fileReadCompletely && inflight.isEmpty()) {
markFileAsDone(reader.getFilePath());
reader = null;
}
super.ack(msgId);
}
private boolean canCommitNow() {
if (commitFrequencyCount > 0 && acksSinceLastCommit >= commitFrequencyCount) {
return true;
}
return commitTimeElapsed.get();
}
@Override
public void fail(Object msgId) {
LOG.trace("Fail received for msg id {} on spout {}", msgId, spoutId);
super.fail(msgId);
if (ackEnabled) {
HdfsUtils.Pair<MessageId, List<Object>> item = HdfsUtils.Pair.of(msgId, inflight.remove(msgId));
retryList.add(item);
}
}
private FileReader pickNextFile() {
try {
// 1) If there are any abandoned files, pick oldest one
lock = getOldestExpiredLock();
if (lock != null) {
LOG.debug("Spout {} now took over ownership of abandoned FileLock {}", spoutId, lock.getLockFile());
Path file = getFileForLockFile(lock.getLockFile(), sourceDirPath);
String resumeFromOffset = lock.getLastLogEntry().fileOffset;
LOG.info("Resuming processing of abandoned file : {}", file);
return createFileReader(file, resumeFromOffset);
}
// 2) If no abandoned files, then pick oldest file in sourceDirPath, lock it and rename it
Collection<Path> listing = HdfsUtils.listFilesByModificationTime(hdfs, sourceDirPath, 0);
for (Path file : listing) {
if (file.getName().endsWith(inprogressSuffix)) {
continue;
}
if (file.getName().endsWith(ignoreSuffix)) {
continue;
}
lock = FileLock.tryLock(hdfs, file, lockDirPath, spoutId);
if (lock == null) {
LOG.debug("Unable to get FileLock for {}, so skipping it.", file);
continue; // could not lock, so try another file.
}
try {
Path newFile = renameToInProgressFile(file);
FileReader result = createFileReader(newFile);
LOG.info("Processing : {} ", file);
return result;
} catch (Exception e) {
LOG.error("Skipping file " + file, e);
releaseLockAndLog(lock, spoutId);
continue;
}
}
return null;
} catch (IOException e) {
LOG.error("Unable to select next file for consumption " + sourceDirPath, e);
return null;
}
}
/**
* If clocks in sync, then acquires the oldest expired lock Else, on first call, just remembers the oldest expired lock, on next call
* check if the lock is updated. if not updated then acquires the lock
*
* @return a lock object
*/
private FileLock getOldestExpiredLock() throws IOException {
// 1 - acquire lock on dir
DirLock dirlock = DirLock.tryLock(hdfs, lockDirPath);
if (dirlock == null) {
dirlock = DirLock.takeOwnershipIfStale(hdfs, lockDirPath, lockTimeoutSec);
if (dirlock == null) {
LOG.debug("Spout {} could not take over ownership of DirLock for {}", spoutId, lockDirPath);
return null;
}
LOG.debug("Spout {} now took over ownership of abandoned DirLock for {}", spoutId, lockDirPath);
} else {
LOG.debug("Spout {} now owns DirLock for {}", spoutId, lockDirPath);
}
try {
// 2 - if clocks are in sync then simply take ownership of the oldest expired lock
if (clocksInSync) {
return FileLock.acquireOldestExpiredLock(hdfs, lockDirPath, lockTimeoutSec, spoutId);
}
// 3 - if clocks are not in sync ..
if (lastExpiredLock == null) {
// just make a note of the oldest expired lock now and check if its still unmodified after lockTimeoutSec
lastExpiredLock = FileLock.locateOldestExpiredLock(hdfs, lockDirPath, lockTimeoutSec);
lastExpiredLockTime = System.currentTimeMillis();
return null;
}
// see if lockTimeoutSec time has elapsed since we last selected the lock file
if (hasExpired(lastExpiredLockTime)) {
return null;
}
// If lock file has expired, then own it
FileLock.LogEntry lastEntry = FileLock.getLastEntry(hdfs, lastExpiredLock.getKey());
if (lastEntry.equals(lastExpiredLock.getValue())) {
FileLock result = FileLock.takeOwnership(hdfs, lastExpiredLock.getKey(), lastEntry, spoutId);
lastExpiredLock = null;
return result;
} else {
// if lock file has been updated since last time, then leave this lock file alone
lastExpiredLock = null;
return null;
}
} finally {
dirlock.release();
LOG.debug("Released DirLock {}, SpoutID {} ", dirlock.getLockFile(), spoutId);
}
}
private boolean hasExpired(long lastModifyTime) {
return (System.currentTimeMillis() - lastModifyTime) < lockTimeoutSec * 1000;
}
/**
* Creates a reader that reads from beginning of file.
*
* @param file file to read
*/
private FileReader createFileReader(Path file)
throws IOException {
if (readerType.equalsIgnoreCase(Configs.SEQ)) {
return new SequenceFileReader(this.hdfs, file, conf);
}
if (readerType.equalsIgnoreCase(Configs.TEXT)) {
return new TextFileReader(this.hdfs, file, conf);
}
try {
Class<?> clsType = Class.forName(readerType);
Constructor<?> constructor = clsType.getConstructor(FileSystem.class, Path.class, Map.class);
return (FileReader) constructor.newInstance(this.hdfs, file, conf);
} catch (Exception e) {
LOG.error(e.getMessage(), e);
throw new RuntimeException("Unable to instantiate " + readerType + " reader", e);
}
}
/**
* Creates a reader that starts reading from 'offset'.
*
* @param file the file to read
* @param offset the offset string should be understandable by the reader type being used
*/
private FileReader createFileReader(Path file, String offset)
throws IOException {
if (readerType.equalsIgnoreCase(Configs.SEQ)) {
return new SequenceFileReader(this.hdfs, file, conf, offset);
}
if (readerType.equalsIgnoreCase(Configs.TEXT)) {
return new TextFileReader(this.hdfs, file, conf, offset);
}
try {
Class<?> clsType = Class.forName(readerType);
Constructor<?> constructor = clsType.getConstructor(FileSystem.class, Path.class, Map.class, String.class);
return (FileReader) constructor.newInstance(this.hdfs, file, conf, offset);
} catch (Exception e) {
LOG.error(e.getMessage(), e);
throw new RuntimeException("Unable to instantiate " + readerType, e);
}
}
/**
* Renames files with .inprogress suffix.
*
* @return path of renamed file
* @throws if operation fails
*/
private Path renameToInProgressFile(Path file)
throws IOException {
Path newFile = new Path(file.toString() + inprogressSuffix);
try {
if (hdfs.rename(file, newFile)) {
return newFile;
}
throw new RenameException(file, newFile);
} catch (IOException e) {
throw new RenameException(file, newFile, e);
}
}
/**
* Returns the corresponding input file in the 'sourceDirPath' for the specified lock file. If no such file is found then returns null
*/
private Path getFileForLockFile(Path lockFile, Path sourceDirPath)
throws IOException {
String lockFileName = lockFile.getName();
Path dataFile = new Path(sourceDirPath + Path.SEPARATOR + lockFileName + inprogressSuffix);
if (hdfs.exists(dataFile)) {
return dataFile;
}
dataFile = new Path(sourceDirPath + Path.SEPARATOR + lockFileName);
if (hdfs.exists(dataFile)) {
return dataFile;
}
return null;
}
// renames files and returns the new file path
private Path renameCompletedFile(Path file) throws IOException {
String fileName = file.toString();
String fileNameMinusSuffix = fileName.substring(0, fileName.indexOf(inprogressSuffix));
String newName = new Path(fileNameMinusSuffix).getName();
Path newFile = new Path(archiveDirPath + Path.SEPARATOR + newName);
LOG.info("Completed consuming file {}", fileNameMinusSuffix);
if (!hdfs.rename(file, newFile)) {
throw new IOException("Rename failed for file: " + file);
}
LOG.debug("Renamed file {} to {} ", file, newFile);
return newFile;
}
@Override
public void declareOutputFields(OutputFieldsDeclarer declarer) {
if (outputStreamName != null) {
declarer.declareStream(outputStreamName, outputFields);
} else {
declarer.declare(outputFields);
}
}
static class MessageId implements Comparable<MessageId> {
public long msgNumber; // tracks order in which msg came in
public String fullPath;
public FileOffset offset;
MessageId(long msgNumber, Path fullPath, FileOffset offset) {
this.msgNumber = msgNumber;
this.fullPath = fullPath.toString();
this.offset = offset;
}
@Override
public String toString() {
return "{'" + fullPath + "':" + offset + "}";
}
@Override
public int compareTo(MessageId rhs) {
if (msgNumber < rhs.msgNumber) {
return -1;
}
if (msgNumber > rhs.msgNumber) {
return 1;
}
return 0;
}
}
private static class RenameException extends IOException {
public final Path oldFile;
public final Path newFile;
RenameException(Path oldFile, Path newFile) {
super("Rename of " + oldFile + " to " + newFile + " failed");
this.oldFile = oldFile;
this.newFile = newFile;
}
RenameException(Path oldFile, Path newFile, IOException cause) {
super("Rename of " + oldFile + " to " + newFile + " failed", cause);
this.oldFile = oldFile;
this.newFile = newFile;
}
}
}