-
Notifications
You must be signed in to change notification settings - Fork 2.6k
/
PersistentProvenanceRepository.java
2446 lines (2088 loc) · 111 KB
/
PersistentProvenanceRepository.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
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
/*
* 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
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* 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.nifi.provenance;
import java.io.EOFException;
import java.io.File;
import java.io.FileFilter;
import java.io.FileNotFoundException;
import java.io.FilenameFilter;
import java.io.IOException;
import java.nio.file.Files;
import java.nio.file.Path;
import java.nio.file.Paths;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.Comparator;
import java.util.Date;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Iterator;
import java.util.LinkedHashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.SortedMap;
import java.util.TreeMap;
import java.util.concurrent.BlockingQueue;
import java.util.concurrent.Callable;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.Future;
import java.util.concurrent.LinkedBlockingQueue;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.ThreadFactory;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicLong;
import java.util.concurrent.atomic.AtomicReference;
import java.util.concurrent.locks.Lock;
import java.util.concurrent.locks.ReadWriteLock;
import java.util.concurrent.locks.ReentrantReadWriteLock;
import java.util.regex.Pattern;
import org.apache.lucene.document.Document;
import org.apache.lucene.index.DirectoryReader;
import org.apache.lucene.index.IndexNotFoundException;
import org.apache.lucene.index.IndexWriter;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.ScoreDoc;
import org.apache.lucene.search.TopDocs;
import org.apache.lucene.store.FSDirectory;
import org.apache.nifi.events.EventReporter;
import org.apache.nifi.processor.DataUnit;
import org.apache.nifi.provenance.expiration.ExpirationAction;
import org.apache.nifi.provenance.expiration.FileRemovalAction;
import org.apache.nifi.provenance.lineage.FlowFileLineage;
import org.apache.nifi.provenance.lineage.Lineage;
import org.apache.nifi.provenance.lineage.LineageComputationType;
import org.apache.nifi.provenance.lucene.DeleteIndexAction;
import org.apache.nifi.provenance.lucene.FieldNames;
import org.apache.nifi.provenance.lucene.IndexManager;
import org.apache.nifi.provenance.lucene.IndexSearch;
import org.apache.nifi.provenance.lucene.IndexingAction;
import org.apache.nifi.provenance.lucene.LineageQuery;
import org.apache.nifi.provenance.lucene.LuceneUtil;
import org.apache.nifi.provenance.search.Query;
import org.apache.nifi.provenance.search.QueryResult;
import org.apache.nifi.provenance.search.QuerySubmission;
import org.apache.nifi.provenance.search.SearchableField;
import org.apache.nifi.provenance.serialization.RecordReader;
import org.apache.nifi.provenance.serialization.RecordReaders;
import org.apache.nifi.provenance.serialization.RecordWriter;
import org.apache.nifi.provenance.serialization.RecordWriters;
import org.apache.nifi.provenance.toc.TocReader;
import org.apache.nifi.provenance.toc.TocUtil;
import org.apache.nifi.reporting.Severity;
import org.apache.nifi.util.FormatUtils;
import org.apache.nifi.util.NiFiProperties;
import org.apache.nifi.util.RingBuffer;
import org.apache.nifi.util.RingBuffer.ForEachEvaluator;
import org.apache.nifi.util.StopWatch;
import org.apache.nifi.util.Tuple;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
public class PersistentProvenanceRepository implements ProvenanceEventRepository {
public static final String DEPRECATED_CLASS_NAME = "nifi.controller.repository.provenance.PersistentProvenanceRepository";
public static final String EVENT_CATEGORY = "Provenance Repository";
private static final String FILE_EXTENSION = ".prov";
private static final String TEMP_FILE_SUFFIX = ".prov.part";
private static final long PURGE_EVENT_MILLISECONDS = 2500L; //Determines the frequency over which the task to delete old events will occur
public static final int SERIALIZATION_VERSION = 8;
public static final Pattern NUMBER_PATTERN = Pattern.compile("\\d+");
public static final Pattern INDEX_PATTERN = Pattern.compile("index-\\d+");
public static final Pattern LOG_FILENAME_PATTERN = Pattern.compile("(\\d+).*\\.prov");
public static final int MAX_UNDELETED_QUERY_RESULTS = 10;
private static final Logger logger = LoggerFactory.getLogger(PersistentProvenanceRepository.class);
private final long maxPartitionMillis;
private final long maxPartitionBytes;
private final AtomicLong idGenerator = new AtomicLong(0L);
private final AtomicReference<SortedMap<Long, Path>> idToPathMap = new AtomicReference<>();
private final AtomicBoolean recoveryFinished = new AtomicBoolean(false);
private final AtomicBoolean closed = new AtomicBoolean(false);
private volatile long firstEventTimestamp = 0L;
// the following are all protected by the lock
private final ReadWriteLock rwLock = new ReentrantReadWriteLock(true);
private final Lock writeLock = rwLock.writeLock();
private final Lock readLock = rwLock.readLock();
private RecordWriter[] writers; // guarded by readLock/writeLock
private final AtomicLong streamStartTime = new AtomicLong(System.currentTimeMillis());
private final RepositoryConfiguration configuration;
private final IndexConfiguration indexConfig;
private final IndexManager indexManager;
private final boolean alwaysSync;
private final int rolloverCheckMillis;
private final int maxAttributeChars;
private final ScheduledExecutorService scheduledExecService;
private final ScheduledExecutorService rolloverExecutor;
private final ExecutorService queryExecService;
private final List<ExpirationAction> expirationActions = new ArrayList<>();
private final ConcurrentMap<String, AsyncQuerySubmission> querySubmissionMap = new ConcurrentHashMap<>();
private final ConcurrentMap<String, AsyncLineageSubmission> lineageSubmissionMap = new ConcurrentHashMap<>();
private final AtomicLong writerIndex = new AtomicLong(0L);
private final AtomicLong storageDirectoryIndex = new AtomicLong(0L);
private final AtomicLong bytesWrittenSinceRollover = new AtomicLong(0L);
private final AtomicInteger recordsWrittenSinceRollover = new AtomicInteger(0);
private final AtomicInteger rolloverCompletions = new AtomicInteger(0);
private final AtomicBoolean initialized = new AtomicBoolean(false);
private final AtomicInteger dirtyWriterCount = new AtomicInteger(0);
// we keep the last 1000 records on hand so that when the UI is opened and it asks for the last 1000 records we don't need to
// read them. Since this is a very cheap operation to keep them, it's worth the tiny expense for the improved user experience.
private final RingBuffer<ProvenanceEventRecord> latestRecords = new RingBuffer<>(1000);
private EventReporter eventReporter;
public PersistentProvenanceRepository() throws IOException {
this(createRepositoryConfiguration(), 10000);
}
public PersistentProvenanceRepository(final RepositoryConfiguration configuration, final int rolloverCheckMillis) throws IOException {
if (configuration.getStorageDirectories().isEmpty()) {
throw new IllegalArgumentException("Must specify at least one storage directory");
}
this.configuration = configuration;
this.maxAttributeChars = configuration.getMaxAttributeChars();
for (final File file : configuration.getStorageDirectories()) {
final Path storageDirectory = file.toPath();
final Path journalDirectory = storageDirectory.resolve("journals");
if (!Files.exists(journalDirectory)) {
Files.createDirectories(journalDirectory);
} else if (!Files.isDirectory(journalDirectory)) {
throw new IllegalArgumentException("Storage Location " + journalDirectory + " is not a directory");
}
}
this.maxPartitionMillis = configuration.getMaxEventFileLife(TimeUnit.MILLISECONDS);
this.maxPartitionBytes = configuration.getMaxEventFileCapacity();
this.indexConfig = new IndexConfiguration(configuration);
this.indexManager = new IndexManager();
this.alwaysSync = configuration.isAlwaysSync();
this.rolloverCheckMillis = rolloverCheckMillis;
scheduledExecService = Executors.newScheduledThreadPool(3, new NamedThreadFactory("Provenance Maintenance Thread"));
queryExecService = Executors.newFixedThreadPool(configuration.getQueryThreadPoolSize(), new NamedThreadFactory("Provenance Query Thread"));
// The number of rollover threads is a little bit arbitrary but comes from the idea that multiple storage directories generally
// live on separate physical partitions. As a result, we want to use at least one thread per partition in order to utilize the
// disks efficiently. However, the rollover actions can be somewhat CPU intensive, so we double the number of threads in order
// to account for that.
final int numRolloverThreads = configuration.getStorageDirectories().size() * 2;
rolloverExecutor = Executors.newScheduledThreadPool(numRolloverThreads, new NamedThreadFactory("Provenance Repository Rollover Thread"));
}
@Override
public void initialize(final EventReporter eventReporter) throws IOException {
writeLock.lock();
try {
if (initialized.getAndSet(true)) {
return;
}
this.eventReporter = eventReporter;
recover();
if (configuration.isAllowRollover()) {
writers = createWriters(configuration, idGenerator.get());
}
if (configuration.isAllowRollover()) {
scheduledExecService.scheduleWithFixedDelay(new Runnable() {
@Override
public void run() {
// Check if we need to roll over
if (needToRollover()) {
// it appears that we do need to roll over. Obtain write lock so that we can do so, and then
// confirm that we still need to.
writeLock.lock();
try {
logger.debug("Obtained write lock to perform periodic rollover");
if (needToRollover()) {
try {
rollover(false);
} catch (final Exception e) {
logger.error("Failed to roll over Provenance Event Log due to {}", e.toString());
logger.error("", e);
eventReporter.reportEvent(Severity.ERROR, EVENT_CATEGORY, "Failed to roll over Provenance Event Log due to " + e.toString());
}
}
} finally {
writeLock.unlock();
}
}
}
}, rolloverCheckMillis, rolloverCheckMillis, TimeUnit.MILLISECONDS);
expirationActions.add(new DeleteIndexAction(this, indexConfig, indexManager));
expirationActions.add(new FileRemovalAction());
scheduledExecService.scheduleWithFixedDelay(new RemoveExpiredQueryResults(), 30L, 3L, TimeUnit.SECONDS);
scheduledExecService.scheduleWithFixedDelay(new Runnable() {
@Override
public void run() {
try {
purgeOldEvents();
} catch (final Exception e) {
logger.error("Failed to purge old events from Provenance Repo due to {}", e.toString());
if (logger.isDebugEnabled()) {
logger.error("", e);
}
eventReporter.reportEvent(Severity.ERROR, EVENT_CATEGORY, "Failed to purge old events from Provenance Repo due to " + e.toString());
}
}
}, PURGE_EVENT_MILLISECONDS, PURGE_EVENT_MILLISECONDS, TimeUnit.MILLISECONDS);
}
firstEventTimestamp = determineFirstEventTimestamp();
} finally {
writeLock.unlock();
}
}
private static RepositoryConfiguration createRepositoryConfiguration() throws IOException {
final NiFiProperties properties = NiFiProperties.getInstance();
final Map<String, Path> storageDirectories = properties.getProvenanceRepositoryPaths();
if (storageDirectories.isEmpty()) {
storageDirectories.put("provenance_repository", Paths.get("provenance_repository"));
}
final String storageTime = properties.getProperty(NiFiProperties.PROVENANCE_MAX_STORAGE_TIME, "24 hours");
final String storageSize = properties.getProperty(NiFiProperties.PROVENANCE_MAX_STORAGE_SIZE, "1 GB");
final String rolloverTime = properties.getProperty(NiFiProperties.PROVENANCE_ROLLOVER_TIME, "5 mins");
final String rolloverSize = properties.getProperty(NiFiProperties.PROVENANCE_ROLLOVER_SIZE, "100 MB");
final String shardSize = properties.getProperty(NiFiProperties.PROVENANCE_INDEX_SHARD_SIZE, "500 MB");
final int queryThreads = properties.getIntegerProperty(NiFiProperties.PROVENANCE_QUERY_THREAD_POOL_SIZE, 2);
final int indexThreads = properties.getIntegerProperty(NiFiProperties.PROVENANCE_INDEX_THREAD_POOL_SIZE, 1);
final int journalCount = properties.getIntegerProperty(NiFiProperties.PROVENANCE_JOURNAL_COUNT, 16);
final long storageMillis = FormatUtils.getTimeDuration(storageTime, TimeUnit.MILLISECONDS);
final long maxStorageBytes = DataUnit.parseDataSize(storageSize, DataUnit.B).longValue();
final long rolloverMillis = FormatUtils.getTimeDuration(rolloverTime, TimeUnit.MILLISECONDS);
final long rolloverBytes = DataUnit.parseDataSize(rolloverSize, DataUnit.B).longValue();
final boolean compressOnRollover = Boolean.parseBoolean(properties.getProperty(NiFiProperties.PROVENANCE_COMPRESS_ON_ROLLOVER));
final String indexedFieldString = properties.getProperty(NiFiProperties.PROVENANCE_INDEXED_FIELDS);
final String indexedAttrString = properties.getProperty(NiFiProperties.PROVENANCE_INDEXED_ATTRIBUTES);
final Boolean alwaysSync = Boolean.parseBoolean(properties.getProperty("nifi.provenance.repository.always.sync", "false"));
final int defaultMaxAttrChars = 65536;
final String maxAttrLength = properties.getProperty("nifi.provenance.repository.max.attribute.length", String.valueOf(defaultMaxAttrChars));
int maxAttrChars;
try {
maxAttrChars = Integer.parseInt(maxAttrLength);
// must be at least 36 characters because that's the length of the uuid attribute,
// which must be kept intact
if (maxAttrChars < 36) {
maxAttrChars = 36;
logger.warn("Found max attribute length property set to " + maxAttrLength + " but minimum length is 36; using 36 instead");
}
} catch (final Exception e) {
maxAttrChars = defaultMaxAttrChars;
}
final List<SearchableField> searchableFields = SearchableFieldParser.extractSearchableFields(indexedFieldString, true);
final List<SearchableField> searchableAttributes = SearchableFieldParser.extractSearchableFields(indexedAttrString, false);
// We always want to index the Event Time.
if (!searchableFields.contains(SearchableFields.EventTime)) {
searchableFields.add(SearchableFields.EventTime);
}
final RepositoryConfiguration config = new RepositoryConfiguration();
for (final Path path : storageDirectories.values()) {
config.addStorageDirectory(path.toFile());
}
config.setCompressOnRollover(compressOnRollover);
config.setSearchableFields(searchableFields);
config.setSearchableAttributes(searchableAttributes);
config.setMaxEventFileCapacity(rolloverBytes);
config.setMaxEventFileLife(rolloverMillis, TimeUnit.MILLISECONDS);
config.setMaxRecordLife(storageMillis, TimeUnit.MILLISECONDS);
config.setMaxStorageCapacity(maxStorageBytes);
config.setQueryThreadPoolSize(queryThreads);
config.setIndexThreadPoolSize(indexThreads);
config.setJournalCount(journalCount);
config.setMaxAttributeChars(maxAttrChars);
if (shardSize != null) {
config.setDesiredIndexSize(DataUnit.parseDataSize(shardSize, DataUnit.B).longValue());
}
config.setAlwaysSync(alwaysSync);
return config;
}
// protected in order to override for unit tests
protected RecordWriter[] createWriters(final RepositoryConfiguration config, final long initialRecordId) throws IOException {
final List<File> storageDirectories = config.getStorageDirectories();
final RecordWriter[] writers = new RecordWriter[config.getJournalCount()];
for (int i = 0; i < config.getJournalCount(); i++) {
final File storageDirectory = storageDirectories.get(i % storageDirectories.size());
final File journalDirectory = new File(storageDirectory, "journals");
final File journalFile = new File(journalDirectory, String.valueOf(initialRecordId) + ".journal." + i);
writers[i] = RecordWriters.newRecordWriter(journalFile, false, false);
writers[i].writeHeader(initialRecordId);
}
logger.info("Created new Provenance Event Writers for events starting with ID {}", initialRecordId);
return writers;
}
/**
* @return the maximum number of characters that any Event attribute should contain. If the event contains
* more characters than this, the attribute may be truncated on retrieval
*/
public int getMaxAttributeCharacters() {
return maxAttributeChars;
}
@Override
public StandardProvenanceEventRecord.Builder eventBuilder() {
return new StandardProvenanceEventRecord.Builder();
}
@Override
public void registerEvent(final ProvenanceEventRecord event) {
persistRecord(Collections.singleton(event));
}
@Override
public void registerEvents(final Iterable<ProvenanceEventRecord> events) {
persistRecord(events);
}
@Override
public List<ProvenanceEventRecord> getEvents(final long firstRecordId, final int maxRecords) throws IOException {
final List<ProvenanceEventRecord> records = new ArrayList<>(maxRecords);
final List<Path> paths = getPathsForId(firstRecordId);
if (paths == null || paths.isEmpty()) {
return records;
}
for (final Path path : paths) {
try (RecordReader reader = RecordReaders.newRecordReader(path.toFile(), getAllLogFiles(), maxAttributeChars)) {
// if this is the first record, try to find out the block index and jump directly to
// the block index. This avoids having to read through a lot of data that we don't care about
// just to get to the first record that we want.
if ( records.isEmpty() ) {
final TocReader tocReader = reader.getTocReader();
if ( tocReader != null ) {
final Integer blockIndex = tocReader.getBlockIndexForEventId(firstRecordId);
if (blockIndex != null) {
reader.skipToBlock(blockIndex);
}
}
}
StandardProvenanceEventRecord record;
while (records.size() < maxRecords && (record = reader.nextRecord()) != null) {
if (record.getEventId() >= firstRecordId) {
records.add(record);
}
}
} catch (final EOFException | FileNotFoundException fnfe) {
// assume file aged off (or there's no data in file, in case of EOFException, which indicates that data was cached
// in operating system and entire O/S crashed and always.sync was not turned on.)
} catch (final IOException ioe) {
logger.error("Failed to read Provenance Event File {} due to {}", path.toFile(), ioe.toString());
logger.error("", ioe);
eventReporter.reportEvent(Severity.ERROR, EVENT_CATEGORY, "Failed to read Provenance Event File " + path.toFile() + " due to " + ioe.toString());
}
if (records.size() >= maxRecords) {
break;
}
}
if (logger.isDebugEnabled()) {
logger.debug("Retrieving up to {} records starting at Event ID {}; returning {} events", maxRecords, firstRecordId, records.size());
}
return records;
}
private List<Path> getPathsForId(final long id) {
final SortedMap<Long, Path> map = idToPathMap.get();
final List<Path> paths = new ArrayList<>();
final Iterator<Map.Entry<Long, Path>> itr = map.entrySet().iterator();
if (!itr.hasNext()) {
return paths;
}
Map.Entry<Long, Path> lastEntry = itr.next();
while (itr.hasNext()) {
final Map.Entry<Long, Path> entry = itr.next();
final Long startIndex = entry.getKey();
if (startIndex >= id) {
paths.add(lastEntry.getValue());
paths.add(entry.getValue());
while (itr.hasNext()) {
paths.add(itr.next().getValue());
}
return paths;
}
lastEntry = entry;
}
// we didn't find any entry whose first ID is greater than the id
// requested. However,
// since we don't know the max ID of the last entry in the map, it's
// possible that the
// ID that we want lives within that file, so we add it to the paths to
// return
if (lastEntry != null) {
paths.add(lastEntry.getValue());
}
if (logger.isDebugEnabled()) {
logger.debug("Looking for Event ID {}, searching in paths: {}", id, paths);
}
return paths;
}
public RepositoryConfiguration getConfiguration() {
return configuration;
}
private void recover() throws IOException {
long maxId = -1L;
long maxIndexedId = -1L;
long minIndexedId = Long.MAX_VALUE;
final List<File> filesToRecover = new ArrayList<>();
for (final File file : configuration.getStorageDirectories()) {
final File[] matchingFiles = file.listFiles(new FileFilter() {
@Override
public boolean accept(final File pathname) {
final String filename = pathname.getName();
if (!filename.contains(FILE_EXTENSION) || filename.endsWith(TEMP_FILE_SUFFIX)) {
return false;
}
final String baseFilename = filename.substring(0, filename.indexOf("."));
return NUMBER_PATTERN.matcher(baseFilename).matches();
}
});
for (final File matchingFile : matchingFiles) {
filesToRecover.add(matchingFile);
}
}
final SortedMap<Long, Path> sortedPathMap = new TreeMap<>(new Comparator<Long>() {
@Override
public int compare(final Long o1, final Long o2) {
return Long.compare(o1, o2);
}
});
File maxIdFile = null;
for (final File file : filesToRecover) {
final String filename = file.getName();
final String baseName = filename.substring(0, filename.indexOf("."));
final long firstId = Long.parseLong(baseName);
sortedPathMap.put(firstId, file.toPath());
if (firstId > maxId) {
maxId = firstId;
maxIdFile = file;
}
if (firstId > maxIndexedId) {
maxIndexedId = firstId - 1;
}
if (firstId < minIndexedId) {
minIndexedId = firstId;
}
}
if (maxIdFile != null) {
// Determine the max ID in the last file.
try (final RecordReader reader = RecordReaders.newRecordReader(maxIdFile, getAllLogFiles(), maxAttributeChars)) {
final long eventId = reader.getMaxEventId();
if (eventId > maxId) {
maxId = eventId;
}
// If the ID is greater than the max indexed id and this file was indexed, then
// update the max indexed id
if (eventId > maxIndexedId) {
maxIndexedId = eventId;
}
} catch (final IOException ioe) {
logger.error("Failed to read Provenance Event File {} due to {}", maxIdFile, ioe);
logger.error("", ioe);
}
}
if (maxIndexedId > -1L) {
// If we have indexed anything then set the min/max ID's indexed.
indexConfig.setMaxIdIndexed(maxIndexedId);
}
if (minIndexedId < Long.MAX_VALUE) {
indexConfig.setMinIdIndexed(minIndexedId);
}
idGenerator.set(maxId + 1);
try {
final Set<File> recoveredJournals = recoverJournalFiles();
filesToRecover.addAll(recoveredJournals);
// Find the file that has the greatest ID
File greatestMinIdFile = null;
long greatestMinId = 0L;
for (final File recoveredJournal : recoveredJournals) {
// if the file was removed because the journals were empty, don't count it
if (!recoveredJournal.exists()) {
continue;
}
final String basename = LuceneUtil.substringBefore(recoveredJournal.getName(), ".");
try {
final long minId = Long.parseLong(basename);
sortedPathMap.put(minId, recoveredJournal.toPath());
if (greatestMinIdFile == null || minId > greatestMinId) {
greatestMinId = minId;
greatestMinIdFile = recoveredJournal;
}
} catch (final NumberFormatException nfe) {
// not a file we care about...
}
}
// Read the records in the last file to find its max id
if (greatestMinIdFile != null) {
try (final RecordReader recordReader = RecordReaders.newRecordReader(greatestMinIdFile, Collections.<Path> emptyList(), maxAttributeChars)) {
maxId = recordReader.getMaxEventId();
}
}
// set the ID Generator 1 greater than the max id
idGenerator.set(maxId + 1);
} catch (final IOException ioe) {
logger.error("Failed to recover Journal Files due to {}", ioe.toString());
logger.error("", ioe);
}
idToPathMap.set(Collections.unmodifiableSortedMap(sortedPathMap));
logger.trace("In recovery, path map: {}", sortedPathMap);
final long recordsRecovered;
if (minIndexedId < Long.MAX_VALUE) {
recordsRecovered = idGenerator.get() - minIndexedId;
} else {
recordsRecovered = idGenerator.get();
}
logger.info("Recovered {} records", recordsRecovered);
recoveryFinished.set(true);
}
@Override
public synchronized void close() throws IOException {
this.closed.set(true);
writeLock.lock();
try {
logger.debug("Obtained write lock for close");
scheduledExecService.shutdownNow();
rolloverExecutor.shutdownNow();
queryExecService.shutdownNow();
indexManager.close();
if ( writers != null ) {
for (final RecordWriter writer : writers) {
writer.close();
}
}
} finally {
writeLock.unlock();
}
}
public boolean isShutdownComplete() {
return this.closed.get();
}
private void persistRecord(final Iterable<ProvenanceEventRecord> records) {
final long totalJournalSize;
readLock.lock();
try {
long bytesWritten = 0L;
// obtain a lock on one of the RecordWriter's so that no other thread is able to write to this writer until we're finished.
// Although the writer itself is thread-safe, we need to generate an event id and then write the event
// atomically, so we need to do this with a lock.
boolean locked = false;
RecordWriter writer;
do {
final RecordWriter[] recordWriters = this.writers;
final int numDirty = dirtyWriterCount.get();
if (numDirty >= recordWriters.length) {
throw new IllegalStateException("Cannot update repository because all partitions are unusable at this time. Writing to the repository would cause corruption. "
+ "This most often happens as a result of the repository running out of disk space or the JMV running out of memory.");
}
final long idx = writerIndex.getAndIncrement();
writer = recordWriters[(int) (idx % recordWriters.length)];
locked = writer.tryLock();
} while (!locked);
try {
try {
for (final ProvenanceEventRecord nextRecord : records) {
final long eventId = idGenerator.getAndIncrement();
bytesWritten += writer.writeRecord(nextRecord, eventId);
logger.trace("Wrote record with ID {} to {}", eventId, writer);
}
if (alwaysSync) {
writer.sync();
}
totalJournalSize = bytesWrittenSinceRollover.addAndGet(bytesWritten);
recordsWrittenSinceRollover.getAndIncrement();
} catch (final Throwable t) {
// We need to set the repoDirty flag before we release the lock for this journal.
// Otherwise, another thread may write to this journal -- this is a problem because
// the journal contains part of our record but not all of it. Writing to the end of this
// journal will result in corruption!
writer.markDirty();
dirtyWriterCount.incrementAndGet();
streamStartTime.set(0L); // force rollover to happen soon.
throw t;
} finally {
writer.unlock();
}
} catch (final IOException ioe) {
// warn about the failure
logger.error("Failed to persist Provenance Event due to {}.", ioe.toString());
logger.error("", ioe);
eventReporter.reportEvent(Severity.ERROR, EVENT_CATEGORY, "Failed to persist Provenance Event due to " + ioe.toString());
// Attempt to perform a rollover. An IOException in this part of the code generally is the result of
// running out of disk space. If we have multiple partitions, we may well be able to rollover. This helps
// in two ways: it compresses the journal files which frees up space, and if it ends up merging to a different
// partition/storage directory, we can delete the journals from this directory that ran out of space.
// In order to do this, though, we must switch from a read lock to a write lock.
// This part of the code gets a little bit messy, and we could potentially refactor it a bit in order to
// make the code cleaner.
readLock.unlock();
try {
writeLock.lock();
try {
logger.debug("Obtained write lock to rollover due to IOException on write");
rollover(true);
} finally {
writeLock.unlock();
}
} catch (final Exception e) {
logger.error("Failed to Rollover Provenance Event Repository file due to {}", e.toString());
logger.error("", e);
eventReporter.reportEvent(Severity.ERROR, EVENT_CATEGORY, "Failed to Rollover Provenance Event Log due to " + e.toString());
} finally {
// we must re-lock the readLock, as the finally block below is going to unlock it.
readLock.lock();
}
return;
}
} finally {
readLock.unlock();
}
// If the total number of bytes written to the Journals is >= configured max, we need to roll over
if (totalJournalSize >= configuration.getMaxEventFileCapacity()) {
writeLock.lock();
try {
logger.debug("Obtained write lock to perform rollover based on file size");
// now that we've obtained the lock, we need to verify that we still need to do the rollover, as
// another thread may have just done it.
if (bytesWrittenSinceRollover.get() >= configuration.getMaxEventFileCapacity()) {
try {
rollover(false);
} catch (final IOException e) {
logger.error("Failed to Rollover Provenance Event Repository file due to {}", e.toString());
logger.error("", e);
eventReporter.reportEvent(Severity.ERROR, EVENT_CATEGORY, "Failed to Rollover Provenance Event Log due to " + e.toString());
}
}
} finally {
writeLock.unlock();
}
}
}
/**
* @return all of the Provenance Event Log Files (not the journals, the merged files) available across all storage directories.
*/
private List<File> getLogFiles() {
final List<File> files = new ArrayList<>();
for (final Path path : idToPathMap.get().values()) {
files.add(path.toFile());
}
if (files.isEmpty()) {
return files;
}
return files;
}
/**
* Returns the size, in bytes, of the Repository storage
*
* @param logFiles the log files to consider
* @param timeCutoff if a log file's last modified date is before timeCutoff, it will be skipped
* @return the size of all log files given whose last mod date comes after (or equal to) timeCutoff
*/
public long getSize(final List<File> logFiles, final long timeCutoff) {
long bytesUsed = 0L;
// calculate the size of the repository
for (final File file : logFiles) {
final long lastModified = file.lastModified();
if (lastModified > 0L && lastModified < timeCutoff) {
continue;
}
bytesUsed += file.length();
}
// take into account the size of the indices
bytesUsed += indexConfig.getIndexSize();
return bytesUsed;
}
/**
* Purges old events from the repository
*
* @throws IOException if unable to purge old events due to an I/O problem
*/
synchronized void purgeOldEvents() throws IOException {
while (!recoveryFinished.get()) {
try {
Thread.sleep(100L);
} catch (final InterruptedException ie) {
}
}
final List<File> toPurge = new ArrayList<>();
final long timeCutoff = System.currentTimeMillis() - configuration.getMaxRecordLife(TimeUnit.MILLISECONDS);
final List<File> sortedByBasename = getLogFiles();
long bytesUsed = getSize(sortedByBasename, timeCutoff);
for (final Path path : idToPathMap.get().values()) {
final File file = path.toFile();
final long lastModified = file.lastModified();
if (lastModified > 0L && lastModified < timeCutoff) {
toPurge.add(file);
}
}
// This comparator sorts the data based on the "basename" of the files. I.e., the numeric portion.
// We do this because the numeric portion represents the ID of the first event in the log file.
// As a result, we are sorting based on time, since the ID is monotonically increasing. By doing this,
// are able to avoid hitting disk continually to check timestamps
final Comparator<File> sortByBasenameComparator = new Comparator<File>() {
@Override
public int compare(final File o1, final File o2) {
final String baseName1 = LuceneUtil.substringBefore(o1.getName(), ".");
final String baseName2 = LuceneUtil.substringBefore(o2.getName(), ".");
Long id1 = null;
Long id2 = null;
try {
id1 = Long.parseLong(baseName1);
} catch (final NumberFormatException nfe) {
id1 = null;
}
try {
id2 = Long.parseLong(baseName2);
} catch (final NumberFormatException nfe) {
id2 = null;
}
if (id1 == null && id2 == null) {
return 0;
}
if (id1 == null) {
return 1;
}
if (id2 == null) {
return -1;
}
return Long.compare(id1, id2);
}
};
// If we have too much data (at least 90% of our max capacity), start aging it off
if (bytesUsed > configuration.getMaxStorageCapacity() * 0.9) {
Collections.sort(sortedByBasename, sortByBasenameComparator);
for (final File file : sortedByBasename) {
toPurge.add(file);
bytesUsed -= file.length();
if (bytesUsed < configuration.getMaxStorageCapacity()) {
// we've shrunk the repo size down enough to stop
break;
}
}
}
// Sort all of the files that we want to purge such that the oldest events are aged off first
Collections.sort(toPurge, sortByBasenameComparator);
logger.debug("Purging old event files: {}", toPurge);
// Remove any duplicates that we may have.
final Set<File> uniqueFilesToPurge = new LinkedHashSet<>(toPurge);
// Age off the data.
final Set<String> removed = new LinkedHashSet<>();
for (File file : uniqueFilesToPurge) {
final String baseName = LuceneUtil.substringBefore(file.getName(), ".");
ExpirationAction currentAction = null;
try {
for (final ExpirationAction action : expirationActions) {
currentAction = action;
if (!action.hasBeenPerformed(file)) {
final File fileBeforeAction = file;
final StopWatch stopWatch = new StopWatch(true);
file = action.execute(file);
stopWatch.stop();
logger.info("Successfully performed Expiration Action {} on Provenance Event file {} in {}", action, fileBeforeAction, stopWatch.getDuration());
}
}
removed.add(baseName);
} catch (final FileNotFoundException fnf) {
logger.warn("Failed to perform Expiration Action {} on Provenance Event file {} because the file no longer exists; will not "
+ "perform additional Expiration Actions on this file", currentAction, file);
removed.add(baseName);
} catch (final Throwable t) {
logger.warn("Failed to perform Expiration Action {} on Provenance Event file {} due to {}; will not perform additional "
+ "Expiration Actions on this file at this time", currentAction, file, t.toString());
logger.warn("", t);
eventReporter.reportEvent(Severity.WARNING, EVENT_CATEGORY, "Failed to perform Expiration Action " + currentAction +
" on Provenance Event file " + file + " due to " + t.toString() + "; will not perform additional Expiration Actions " +
"on this file at this time");
}
}
// Update the Map ID to Path map to not include the removed file
// We cannot obtain the write lock here because there may be a need for the lock in the rollover method,
// if we have 'backpressure applied'. This would result in a deadlock because the rollover method would be
// waiting for purgeOldEvents, and purgeOldEvents would be waiting for the write lock held by rollover.
boolean updated = false;
while (!updated) {
final SortedMap<Long, Path> existingPathMap = idToPathMap.get();
final SortedMap<Long, Path> newPathMap = new TreeMap<>(new PathMapComparator());
newPathMap.putAll(existingPathMap);
final Iterator<Map.Entry<Long, Path>> itr = newPathMap.entrySet().iterator();
while (itr.hasNext()) {
final Map.Entry<Long, Path> entry = itr.next();
final String filename = entry.getValue().toFile().getName();
final String baseName = LuceneUtil.substringBefore(filename, ".");
if (removed.contains(baseName)) {
itr.remove();
}
}
updated = idToPathMap.compareAndSet(existingPathMap, newPathMap);
logger.debug("After expiration, path map: {}", newPathMap);
}
purgeExpiredIndexes();
}
private void purgeExpiredIndexes() throws IOException {
// Now that we have potentially removed expired Provenance Event Log Files, we can look at
// whether or not we can delete any of the indexes. An index can be deleted if all of the
// data that is associated with that index has already been deleted. In order to test this,
// we will get the timestamp of the earliest event and then compare that to the latest timestamp
// that would be indexed by the earliest index. If the event occurred after the timestamp of
// the latest index, then we can just delete the entire index all together.
// find all of the index directories
final List<File> indexDirs = getAllIndexDirectories();
if (indexDirs.size() < 2) {
this.firstEventTimestamp = determineFirstEventTimestamp();
return;
}
// Indexes are named "index-XXX" where the XXX is the timestamp of the earliest event that
// could be in the index. Once we have finished with one index, we move on to another index,
// but we don't move on until we are finished with the previous index.
// Therefore, an efficient way to determine the latest timestamp of one index is to look at the
// timestamp of the next index (these could potentially overlap for one millisecond). This is
// efficient because we can determine the earliest timestamp of an index simply by looking at
// the name of the Index's directory.
final long latestTimestampOfFirstIndex = getIndexTimestamp(indexDirs.get(1));
// Get the timestamp of the first event in the first Provenance Event Log File and the ID of the last event
// in the event file.
final List<File> logFiles = getSortedLogFiles();
if (logFiles.isEmpty()) {
this.firstEventTimestamp = System.currentTimeMillis();
return;
}
final File firstLogFile = logFiles.get(0);
long earliestEventTime = System.currentTimeMillis();
long maxEventId = -1L;
try (final RecordReader reader = RecordReaders.newRecordReader(firstLogFile, null, Integer.MAX_VALUE)) {
final StandardProvenanceEventRecord event = reader.nextRecord();
earliestEventTime = event.getEventTime();
try {
maxEventId = reader.getMaxEventId();
} catch (final IOException ioe) {
logger.warn("Unable to determine the maximum ID for Provenance Event Log File {}; values reported for the number of "
+ "events in the Provenance Repository may be inaccurate.", firstLogFile);
}