-
Notifications
You must be signed in to change notification settings - Fork 962
/
StorageGroupProcessor.java
executable file
·1571 lines (1441 loc) · 59.9 KB
/
StorageGroupProcessor.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.iotdb.db.engine.storagegroup;
import static org.apache.iotdb.db.engine.merge.task.MergeTask.MERGE_SUFFIX;
import static org.apache.iotdb.db.engine.storagegroup.TsFileResource.TEMP_SUFFIX;
import static org.apache.iotdb.tsfile.common.constant.TsFileConstant.TSFILE_SUFFIX;
import java.io.File;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.Date;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Iterator;
import java.util.LinkedList;
import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
import java.util.Set;
import java.util.concurrent.locks.ReadWriteLock;
import java.util.concurrent.locks.ReentrantReadWriteLock;
import org.apache.commons.io.FileUtils;
import org.apache.iotdb.db.conf.IoTDBConstant;
import org.apache.iotdb.db.conf.IoTDBDescriptor;
import org.apache.iotdb.db.conf.directories.DirectoryManager;
import org.apache.iotdb.db.engine.fileSystem.SystemFileFactory;
import org.apache.iotdb.db.engine.flush.TsFileFlushPolicy;
import org.apache.iotdb.db.engine.merge.manage.MergeManager;
import org.apache.iotdb.db.engine.merge.manage.MergeResource;
import org.apache.iotdb.db.engine.merge.selector.IMergeFileSelector;
import org.apache.iotdb.db.engine.merge.selector.MaxFileMergeFileSelector;
import org.apache.iotdb.db.engine.merge.selector.MaxSeriesMergeFileSelector;
import org.apache.iotdb.db.engine.merge.selector.MergeFileStrategy;
import org.apache.iotdb.db.engine.merge.task.MergeTask;
import org.apache.iotdb.db.engine.merge.task.RecoverMergeTask;
import org.apache.iotdb.db.engine.modification.Deletion;
import org.apache.iotdb.db.engine.modification.Modification;
import org.apache.iotdb.db.engine.modification.ModificationFile;
import org.apache.iotdb.db.engine.querycontext.QueryDataSource;
import org.apache.iotdb.db.engine.querycontext.ReadOnlyMemChunk;
import org.apache.iotdb.db.engine.version.SimpleFileVersionController;
import org.apache.iotdb.db.engine.version.VersionController;
import org.apache.iotdb.db.exception.DiskSpaceInsufficientException;
import org.apache.iotdb.db.exception.MergeException;
import org.apache.iotdb.db.exception.TsFileProcessorException;
import org.apache.iotdb.db.exception.metadata.MetadataException;
import org.apache.iotdb.db.exception.query.OutOfTTLException;
import org.apache.iotdb.db.exception.query.QueryProcessException;
import org.apache.iotdb.db.exception.storageGroup.StorageGroupProcessorException;
import org.apache.iotdb.db.metadata.MManager;
import org.apache.iotdb.db.qp.physical.crud.BatchInsertPlan;
import org.apache.iotdb.db.qp.physical.crud.DeletePlan;
import org.apache.iotdb.db.qp.physical.crud.InsertPlan;
import org.apache.iotdb.db.query.context.QueryContext;
import org.apache.iotdb.db.query.control.QueryFileManager;
import org.apache.iotdb.db.utils.CopyOnReadLinkedList;
import org.apache.iotdb.db.utils.TestOnly;
import org.apache.iotdb.db.utils.UpgradeUtils;
import org.apache.iotdb.db.writelog.recover.TsFileRecoverPerformer;
import org.apache.iotdb.rpc.TSStatusCode;
import org.apache.iotdb.tsfile.file.metadata.ChunkMetaData;
import org.apache.iotdb.tsfile.file.metadata.enums.CompressionType;
import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding;
import org.apache.iotdb.tsfile.fileSystem.FSFactoryProducer;
import org.apache.iotdb.tsfile.fileSystem.fsFactory.FSFactory;
import org.apache.iotdb.tsfile.read.common.Path;
import org.apache.iotdb.tsfile.utils.Pair;
import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
import org.apache.iotdb.tsfile.write.schema.Schema;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* For sequence data, a StorageGroupProcessor has some TsFileProcessors, in which there is only one
* TsFileProcessor in the working status. <br/>
*
* There are two situations to set the working TsFileProcessor to closing status:<br/>
*
* (1) when inserting data into the TsFileProcessor, and the TsFileProcessor shouldFlush() (or
* shouldClose())<br/>
*
* (2) someone calls waitForAllCurrentTsFileProcessorsClosed(). (up to now, only flush command from
* cli will call this method)<br/>
*
* UnSequence data has the similar process as above.
*
* When a sequence TsFileProcessor is submitted to be flushed, the updateLatestFlushTimeCallback()
* method will be called as a callback.<br/>
*
* When a TsFileProcessor is closed, the closeUnsealedTsFileProcessor() method will be called as a
* callback.
*/
public class StorageGroupProcessor {
private static final String MERGING_MODIFICATION_FILE_NAME = "merge.mods";
private static final Logger logger = LoggerFactory.getLogger(StorageGroupProcessor.class);
private static final int MAX_CACHE_SENSORS = 5000;
/**
* a read write lock for guaranteeing concurrent safety when accessing all fields in this class
* (i.e., schema, (un)sequenceFileList, work(un)SequenceTsFileProcessor,
* closing(Un)SequenceTsFileProcessor, latestTimeForEachDevice, and
* latestFlushedTimeForEachDevice)
*/
private final ReadWriteLock insertLock = new ReentrantReadWriteLock();
/**
* closeStorageGroupCondition is used to wait for all currently closing TsFiles to be done.
*/
private final Object closeStorageGroupCondition = new Object();
/**
* avoid some tsfileResource is changed (e.g., from unsealed to sealed) when a query is executed.
*/
private final ReadWriteLock closeQueryLock = new ReentrantReadWriteLock();
/**
* the schema of time series that belong this storage group
*/
private Schema schema;
// includes sealed and unsealed sequence TsFiles
private List<TsFileResource> sequenceFileList = new ArrayList<>();
private TsFileProcessor workSequenceTsFileProcessor = null;
private CopyOnReadLinkedList<TsFileProcessor> closingSequenceTsFileProcessor = new CopyOnReadLinkedList<>();
// includes sealed and unsealed unSequence TsFiles
private List<TsFileResource> unSequenceFileList = new ArrayList<>();
private TsFileProcessor workUnSequenceTsFileProcessor = null;
private CopyOnReadLinkedList<TsFileProcessor> closingUnSequenceTsFileProcessor = new CopyOnReadLinkedList<>();
/**
* device -> global latest timestamp of each device latestTimeForEachDevice caches non-flushed
* changes upon timestamps of each device, and is used to update latestFlushedTimeForEachDevice
* when a flush is issued.
*/
private Map<String, Long> latestTimeForEachDevice = new HashMap<>();
/**
* device -> largest timestamp of the latest memtable to be submitted to asyncTryToFlush
* latestFlushedTimeForEachDevice determines whether a data point should be put into a sequential
* file or an unsequential file. Data of some device with timestamp less than or equals to the
* device's latestFlushedTime should go into an unsequential file.
*/
private Map<String, Long> latestFlushedTimeForEachDevice = new HashMap<>();
private String storageGroupName;
private File storageGroupSysDir;
/**
* versionController assigns a version for each MemTable and deletion/update such that after they
* are persisted, the order of insertions, deletions and updates can be re-determined.
*/
private VersionController versionController;
/**
* mergeLock is to be used in the merge process. Concurrent queries, deletions and merges may
* result in losing some deletion in the merged new file, so a lock is necessary.
*/
private ReentrantReadWriteLock mergeLock = new ReentrantReadWriteLock();
/**
* This is the modification file of the result of the current merge. Because the merged file may
* be invisible at this moment, without this, deletion/update during merge could be lost.
*/
private ModificationFile mergingModification;
private volatile boolean isMerging = false;
private long mergeStartTime;
/**
* This linked list records the access order of measurements used by query.
*/
private LinkedList<String> lruForSensorUsedInQuery = new LinkedList<>();
/**
* when the data in a storage group is older than dataTTL, it is considered invalid and will be
* eventually removed.
*/
private long dataTTL = Long.MAX_VALUE;
private FSFactory fsFactory = FSFactoryProducer.getFSFactory();
private TsFileFlushPolicy fileFlushPolicy;
public StorageGroupProcessor(String systemInfoDir, String storageGroupName, TsFileFlushPolicy fileFlushPolicy)
throws StorageGroupProcessorException {
this.storageGroupName = storageGroupName;
this.fileFlushPolicy = fileFlushPolicy;
// construct the file schema
this.schema = constructSchema(storageGroupName);
try {
storageGroupSysDir = SystemFileFactory.INSTANCE.getFile(systemInfoDir, storageGroupName);
if (storageGroupSysDir.mkdirs()) {
logger.info("Storage Group system Directory {} doesn't exist, create it",
storageGroupSysDir.getPath());
} else if (!storageGroupSysDir.exists()) {
logger.error("create Storage Group system Directory {} failed",
storageGroupSysDir.getPath());
}
versionController = new SimpleFileVersionController(storageGroupSysDir.getPath());
} catch (IOException e) {
throw new StorageGroupProcessorException(e);
}
recover();
}
private void recover() throws StorageGroupProcessorException {
logger.info("recover Storage Group {}", storageGroupName);
try {
// collect TsFiles from sequential and unsequential data directory
List<TsFileResource> seqTsFiles = getAllFiles(
DirectoryManager.getInstance().getAllSequenceFileFolders());
List<TsFileResource> unseqTsFiles =
getAllFiles(DirectoryManager.getInstance().getAllUnSequenceFileFolders());
recoverSeqFiles(seqTsFiles);
recoverUnseqFiles(unseqTsFiles);
String taskName = storageGroupName + "-" + System.currentTimeMillis();
File mergingMods = SystemFileFactory.INSTANCE.getFile(storageGroupSysDir,
MERGING_MODIFICATION_FILE_NAME);
if (mergingMods.exists()) {
mergingModification = new ModificationFile(mergingMods.getPath());
}
RecoverMergeTask recoverMergeTask = new RecoverMergeTask(seqTsFiles, unseqTsFiles,
storageGroupSysDir.getPath(), this::mergeEndAction, taskName,
IoTDBDescriptor.getInstance().getConfig().isForceFullMerge(), storageGroupName);
logger.info("{} a RecoverMergeTask {} starts...", storageGroupName, taskName);
recoverMergeTask
.recoverMerge(IoTDBDescriptor.getInstance().getConfig().isContinueMergeAfterReboot());
if (!IoTDBDescriptor.getInstance().getConfig().isContinueMergeAfterReboot()) {
mergingMods.delete();
}
} catch (IOException | MetadataException e) {
throw new StorageGroupProcessorException(e);
}
for (TsFileResource resource : sequenceFileList) {
latestTimeForEachDevice.putAll(resource.getEndTimeMap());
latestFlushedTimeForEachDevice.putAll(resource.getEndTimeMap());
}
}
private List<TsFileResource> getAllFiles(List<String> folders) {
List<File> tsFiles = new ArrayList<>();
for (String baseDir : folders) {
File fileFolder = fsFactory.getFile(baseDir, storageGroupName);
if (!fileFolder.exists()) {
continue;
}
// some TsFileResource may be being persisted when the system crashed, try recovering such
// resources
continueFailedRenames(fileFolder, TEMP_SUFFIX);
// some TsFiles were going to be replaced by the merged files when the system crashed and
// the process was interrupted before the merged files could be named
continueFailedRenames(fileFolder, MERGE_SUFFIX);
Collections.addAll(tsFiles,
fsFactory.listFilesBySuffix(fileFolder.getAbsolutePath(), TSFILE_SUFFIX));
}
tsFiles.sort(this::compareFileName);
List<TsFileResource> ret = new ArrayList<>();
tsFiles.forEach(f -> ret.add(new TsFileResource(f)));
return ret;
}
private void continueFailedRenames(File fileFolder, String suffix) {
File[] files = fsFactory.listFilesBySuffix(fileFolder.getAbsolutePath(), suffix);
if (files != null) {
for (File tempResource : files) {
File originResource = fsFactory.getFile(tempResource.getPath().replace(suffix, ""));
if (originResource.exists()) {
tempResource.delete();
} else {
tempResource.renameTo(originResource);
}
}
}
}
private void recoverSeqFiles(List<TsFileResource> tsFiles) throws StorageGroupProcessorException {
for (TsFileResource tsFileResource : tsFiles) {
sequenceFileList.add(tsFileResource);
TsFileRecoverPerformer recoverPerformer = new TsFileRecoverPerformer(storageGroupName + "-"
, schema, versionController, tsFileResource, false);
recoverPerformer.recover();
tsFileResource.setClosed(true);
}
}
private void recoverUnseqFiles(List<TsFileResource> tsFiles)
throws StorageGroupProcessorException {
for (TsFileResource tsFileResource : tsFiles) {
unSequenceFileList.add(tsFileResource);
TsFileRecoverPerformer recoverPerformer = new TsFileRecoverPerformer(storageGroupName + "-",
schema,
versionController, tsFileResource, true);
recoverPerformer.recover();
tsFileResource.setClosed(true);
}
}
// ({systemTime}-{versionNum}-{mergeNum}.tsfile)
private int compareFileName(File o1, File o2) {
String[] items1 = o1.getName().replace(TSFILE_SUFFIX, "")
.split(IoTDBConstant.TSFILE_NAME_SEPARATOR);
String[] items2 = o2.getName().replace(TSFILE_SUFFIX, "")
.split(IoTDBConstant.TSFILE_NAME_SEPARATOR);
long ver1 = Long.parseLong(items1[0]);
long ver2 = Long.parseLong(items2[0]);
int cmp = Long.compare(ver1, ver2);
if (cmp == 0) {
return Long.compare(Long.parseLong(items1[1]), Long.parseLong(items2[1]));
} else {
return cmp;
}
}
private Schema constructSchema(String storageGroupName) {
List<MeasurementSchema> columnSchemaList;
columnSchemaList = MManager.getInstance().getSchemaForStorageGroup(storageGroupName);
Schema newSchema = new Schema();
for (MeasurementSchema measurementSchema : columnSchemaList) {
newSchema.registerMeasurement(measurementSchema);
}
return newSchema;
}
/**
* add a measurement into the schema.
*/
public void addMeasurement(String measurementId, TSDataType dataType, TSEncoding encoding,
CompressionType compressor, Map<String, String> props) {
writeLock();
try {
schema.registerMeasurement(new MeasurementSchema(measurementId, dataType, encoding,
compressor, props));
} finally {
writeUnlock();
}
}
public void insert(InsertPlan insertPlan) throws QueryProcessException {
// reject insertions that are out of ttl
if (!checkTTL(insertPlan.getTime())) {
throw new OutOfTTLException(insertPlan.getTime(), (System.currentTimeMillis() - dataTTL));
}
writeLock();
try {
// init map
latestTimeForEachDevice.putIfAbsent(insertPlan.getDeviceId(), Long.MIN_VALUE);
latestFlushedTimeForEachDevice.putIfAbsent(insertPlan.getDeviceId(), Long.MIN_VALUE);
// insert to sequence or unSequence file
insertToTsFileProcessor(insertPlan,
insertPlan.getTime() > latestFlushedTimeForEachDevice.get(insertPlan.getDeviceId()));
} finally {
writeUnlock();
}
}
public Integer[] insertBatch(BatchInsertPlan batchInsertPlan) throws QueryProcessException {
writeLock();
try {
// init map
latestTimeForEachDevice.putIfAbsent(batchInsertPlan.getDeviceId(), Long.MIN_VALUE);
latestFlushedTimeForEachDevice.putIfAbsent(batchInsertPlan.getDeviceId(), Long.MIN_VALUE);
Integer[] results = new Integer[batchInsertPlan.getRowCount()];
List<Integer> sequenceIndexes = new ArrayList<>();
List<Integer> unsequenceIndexes = new ArrayList<>();
long lastFlushTime = latestFlushedTimeForEachDevice.get(batchInsertPlan.getDeviceId());
for (int i = 0; i < batchInsertPlan.getRowCount(); i++) {
long currTime = batchInsertPlan.getTimes()[i];
// skip points that do not satisfy TTL
if (!checkTTL(currTime)) {
results[i] = TSStatusCode.OUT_OF_TTL_ERROR.getStatusCode();
continue;
}
results[i] = TSStatusCode.SUCCESS_STATUS.getStatusCode();
if (currTime > lastFlushTime) {
sequenceIndexes.add(i);
} else {
unsequenceIndexes.add(i);
}
}
if (!sequenceIndexes.isEmpty()) {
insertBatchToTsFileProcessor(batchInsertPlan, sequenceIndexes, true, results);
}
if (!unsequenceIndexes.isEmpty()) {
insertBatchToTsFileProcessor(batchInsertPlan, unsequenceIndexes, false, results);
}
return results;
} finally {
writeUnlock();
}
}
/**
* @return whether the given time falls in ttl
*/
private boolean checkTTL(long time) {
return dataTTL == Long.MAX_VALUE || (System.currentTimeMillis() - time) <= dataTTL;
}
private void insertBatchToTsFileProcessor(BatchInsertPlan batchInsertPlan,
List<Integer> indexes, boolean sequence, Integer[] results) throws QueryProcessException {
TsFileProcessor tsFileProcessor = getOrCreateTsFileProcessor(sequence);
if (tsFileProcessor == null) {
for (int index : indexes) {
results[index] = TSStatusCode.INTERNAL_SERVER_ERROR.getStatusCode();
}
return;
}
boolean result = tsFileProcessor.insertBatch(batchInsertPlan, indexes, results);
// try to update the latest time of the device of this tsRecord
if (result && latestTimeForEachDevice.get(batchInsertPlan.getDeviceId()) < batchInsertPlan
.getMaxTime()) {
latestTimeForEachDevice.put(batchInsertPlan.getDeviceId(), batchInsertPlan.getMaxTime());
}
// check memtable size and may asyncTryToFlush the work memtable
if (tsFileProcessor.shouldFlush()) {
fileFlushPolicy.apply(this, tsFileProcessor, sequence);
}
}
private void insertToTsFileProcessor(InsertPlan insertPlan, boolean sequence)
throws QueryProcessException {
TsFileProcessor tsFileProcessor;
boolean result;
tsFileProcessor = getOrCreateTsFileProcessor(sequence);
if (tsFileProcessor == null) {
return;
}
// insert TsFileProcessor
result = tsFileProcessor.insert(insertPlan);
// try to update the latest time of the device of this tsRecord
if (result && latestTimeForEachDevice.get(insertPlan.getDeviceId()) < insertPlan.getTime()) {
latestTimeForEachDevice.put(insertPlan.getDeviceId(), insertPlan.getTime());
}
// check memtable size and may asyncTryToFlush the work memtable
if (tsFileProcessor.shouldFlush()) {
fileFlushPolicy.apply(this, tsFileProcessor, sequence);
}
}
private TsFileProcessor getOrCreateTsFileProcessor(boolean sequence) {
TsFileProcessor tsFileProcessor = null;
try {
if (sequence) {
if (workSequenceTsFileProcessor == null) {
// create a new TsfileProcessor
workSequenceTsFileProcessor = createTsFileProcessor(true);
sequenceFileList.add(workSequenceTsFileProcessor.getTsFileResource());
}
tsFileProcessor = workSequenceTsFileProcessor;
} else {
if (workUnSequenceTsFileProcessor == null) {
// create a new TsfileProcessor
workUnSequenceTsFileProcessor = createTsFileProcessor(false);
unSequenceFileList.add(workUnSequenceTsFileProcessor.getTsFileResource());
}
tsFileProcessor = workUnSequenceTsFileProcessor;
}
} catch (DiskSpaceInsufficientException e) {
logger.error(
"disk space is insufficient when creating TsFile processor, change system mode to read-only",
e);
IoTDBDescriptor.getInstance().getConfig().setReadOnly(true);
} catch (IOException e) {
logger
.error("meet IOException when creating TsFileProcessor, change system mode to read-only",
e);
IoTDBDescriptor.getInstance().getConfig().setReadOnly(true);
}
return tsFileProcessor;
}
private TsFileProcessor createTsFileProcessor(boolean sequence)
throws IOException, DiskSpaceInsufficientException {
String baseDir;
if (sequence) {
baseDir = DirectoryManager.getInstance().getNextFolderForSequenceFile();
} else {
baseDir = DirectoryManager.getInstance().getNextFolderForUnSequenceFile();
}
fsFactory.getFile(baseDir, storageGroupName).mkdirs();
String filePath = baseDir + File.separator + storageGroupName + File.separator +
System.currentTimeMillis() + IoTDBConstant.TSFILE_NAME_SEPARATOR + versionController
.nextVersion() + IoTDBConstant.TSFILE_NAME_SEPARATOR + "0" + TSFILE_SUFFIX;
if (sequence) {
return new TsFileProcessor(storageGroupName, fsFactory.getFile(filePath),
schema, versionController, this::closeUnsealedTsFileProcessor,
this::updateLatestFlushTimeCallback, sequence);
} else {
return new TsFileProcessor(storageGroupName, fsFactory.getFile(filePath),
schema, versionController, this::closeUnsealedTsFileProcessor,
() -> true, sequence);
}
}
/**
* thread-safety should be ensured by caller
*/
public void moveOneWorkProcessorToClosingList(boolean sequence) {
//for sequence tsfile, we update the endTimeMap only when the file is prepared to be closed.
//for unsequence tsfile, we have maintained the endTimeMap when an insertion comes.
if (sequence && workSequenceTsFileProcessor != null) {
closingSequenceTsFileProcessor.add(workSequenceTsFileProcessor);
updateEndTimeMap(workSequenceTsFileProcessor);
workSequenceTsFileProcessor.asyncClose();
workSequenceTsFileProcessor = null;
logger.info("close a sequence tsfile processor {}", storageGroupName);
} else if (workUnSequenceTsFileProcessor != null){
closingUnSequenceTsFileProcessor.add(workUnSequenceTsFileProcessor);
workUnSequenceTsFileProcessor.asyncClose();
workUnSequenceTsFileProcessor = null;
logger.info("close an unsequence tsfile processor {}", storageGroupName);
}
}
/**
* delete the storageGroup's own folder in folder data/system/storage_groups
*/
public void deleteFolder(String systemDir) {
waitForAllCurrentTsFileProcessorsClosed();
writeLock();
try {
File storageGroupFolder = SystemFileFactory.INSTANCE.getFile(systemDir, storageGroupName);
if (storageGroupFolder.exists()) {
FileUtils.deleteDirectory(storageGroupFolder);
}
} catch (IOException e) {
logger.error("Cannot delete the folder in storage group {}, because", storageGroupName, e);
} finally {
writeUnlock();
}
}
public void syncDeleteDataFiles() {
waitForAllCurrentTsFileProcessorsClosed();
writeLock();
try {
for (TsFileResource tsFileResource : unSequenceFileList) {
tsFileResource.close();
}
for (TsFileResource tsFileResource : sequenceFileList) {
tsFileResource.close();
}
List<String> folder = DirectoryManager.getInstance().getAllSequenceFileFolders();
folder.addAll(DirectoryManager.getInstance().getAllUnSequenceFileFolders());
deleteAllSGFolders(folder);
this.workSequenceTsFileProcessor = null;
this.workUnSequenceTsFileProcessor = null;
this.sequenceFileList.clear();
this.unSequenceFileList.clear();
this.latestFlushedTimeForEachDevice.clear();
this.latestTimeForEachDevice.clear();
} catch (IOException e) {
logger.error("Cannot delete files in storage group {}", storageGroupName, e);
} finally {
writeUnlock();
}
}
private void deleteAllSGFolders(List<String> folder) {
for (String tsfilePath : folder) {
File storageGroupFolder = fsFactory.getFile(tsfilePath, storageGroupName);
if (storageGroupFolder.exists()) {
try {
FileUtils.deleteDirectory(storageGroupFolder);
} catch (IOException e) {
logger.error("Delete TsFiles failed", e);
}
}
}
}
/**
* Iterate each TsFile and try to lock and remove those out of TTL.
*/
public synchronized void checkFilesTTL() {
if (dataTTL == Long.MAX_VALUE) {
logger.debug("{}: TTL not set, ignore the check", storageGroupName);
return;
}
long timeLowerBound = System.currentTimeMillis() - dataTTL;
if (logger.isDebugEnabled()) {
logger.debug("{}: TTL removing files before {}", storageGroupName, new Date(timeLowerBound));
}
// copy to avoid concurrent modification of deletion
List<TsFileResource> seqFiles = new ArrayList<>(sequenceFileList);
List<TsFileResource> unseqFiles = new ArrayList<>(unSequenceFileList);
for (TsFileResource tsFileResource : seqFiles) {
checkFileTTL(tsFileResource, timeLowerBound, true);
}
for (TsFileResource tsFileResource : unseqFiles) {
checkFileTTL(tsFileResource, timeLowerBound, false);
}
}
private void checkFileTTL(TsFileResource resource, long timeLowerBound, boolean isSeq) {
if (resource.isMerging() || !resource.isClosed()
|| !resource.isDeleted() && resource.stillLives(timeLowerBound)) {
return;
}
writeLock();
try {
// prevent new merges and queries from choosing this file
resource.setDeleted(true);
// the file may be chosen for merge after the last check and before writeLock()
// double check to ensure the file is not used by a merge
if (resource.isMerging()) {
return;
}
// ensure that the file is not used by any queries
if (resource.getWriteQueryLock().writeLock().tryLock()) {
try {
// physical removal
resource.remove();
if (logger.isInfoEnabled()) {
logger.info("Removed a file {} before {} by ttl ({}ms)", resource.getFile().getPath(),
new Date(timeLowerBound), dataTTL);
}
if (isSeq) {
sequenceFileList.remove(resource);
} else {
unSequenceFileList.remove(resource);
}
} finally {
resource.getWriteQueryLock().writeLock().unlock();
}
}
} finally {
writeUnlock();
}
}
/**
* This method will be blocked until all tsfile processors are closed.
*/
public void waitForAllCurrentTsFileProcessorsClosed() {
synchronized (closeStorageGroupCondition) {
try {
putAllWorkingTsFileProcessorIntoClosingList();
while (!closingSequenceTsFileProcessor.isEmpty() || !closingUnSequenceTsFileProcessor
.isEmpty()) {
closeStorageGroupCondition.wait();
}
} catch (InterruptedException e) {
logger.error("CloseFileNodeCondition error occurs while waiting for closing the storage "
+ "group {}", storageGroupName, e);
}
}
}
public void putAllWorkingTsFileProcessorIntoClosingList() {
writeLock();
try {
logger.info("async force close all files in storage group: {}", storageGroupName);
if (workSequenceTsFileProcessor != null) {
moveOneWorkProcessorToClosingList(true);
}
if (workUnSequenceTsFileProcessor != null) {
moveOneWorkProcessorToClosingList(false);
}
} finally {
writeUnlock();
}
}
// TODO need a read lock, please consider the concurrency with flush manager threads.
public QueryDataSource query(String deviceId, String measurementId, QueryContext context,
QueryFileManager filePathsManager) {
insertLock.readLock().lock();
mergeLock.readLock().lock();
synchronized (lruForSensorUsedInQuery) {
if (lruForSensorUsedInQuery.size() >= MAX_CACHE_SENSORS) {
lruForSensorUsedInQuery.removeFirst();
}
lruForSensorUsedInQuery.add(measurementId);
}
try {
List<TsFileResource> seqResources = getFileReSourceListForQuery(sequenceFileList,
deviceId, measurementId, context);
List<TsFileResource> unseqResources = getFileReSourceListForQuery(unSequenceFileList,
deviceId, measurementId, context);
QueryDataSource dataSource = new QueryDataSource(new Path(deviceId, measurementId),
seqResources, unseqResources);
// used files should be added before mergeLock is unlocked, or they may be deleted by
// running merge
// is null only in tests
if (filePathsManager != null) {
filePathsManager.addUsedFilesForQuery(context.getQueryId(), dataSource);
}
dataSource.setDataTTL(dataTTL);
return dataSource;
} finally {
insertLock.readLock().unlock();
mergeLock.readLock().unlock();
}
}
/**
* returns the top k% measurements which are recently used in queries.
*/
public Set calTopKMeasurement(String sensorId, double k) {
int num = (int) (lruForSensorUsedInQuery.size() * k);
Set<String> sensorSet = new HashSet<>(num + 1);
synchronized (lruForSensorUsedInQuery) {
Iterator<String> iterator = lruForSensorUsedInQuery.descendingIterator();
while (iterator.hasNext() && sensorSet.size() < num) {
String sensor = iterator.next();
if (sensorSet.contains(sensor)) {
iterator.remove();
} else {
sensorSet.add(sensor);
}
}
}
sensorSet.add(sensorId);
return sensorSet;
}
public void writeLock() {
insertLock.writeLock().lock();
}
public void writeUnlock() {
insertLock.writeLock().unlock();
}
/**
* @param tsFileResources includes sealed and unsealed tsfile resources
* @return fill unsealed tsfile resources with memory data and ChunkMetadataList of data in disk
*/
private List<TsFileResource> getFileReSourceListForQuery(List<TsFileResource> tsFileResources,
String deviceId, String measurementId, QueryContext context) {
MeasurementSchema mSchema = schema.getMeasurementSchema(measurementId);
TSDataType dataType = mSchema.getType();
List<TsFileResource> tsfileResourcesForQuery = new ArrayList<>();
long timeLowerBound = dataTTL != Long.MAX_VALUE ? System.currentTimeMillis() - dataTTL : Long
.MIN_VALUE;
context.setQueryTimeLowerBound(timeLowerBound);
for (TsFileResource tsFileResource : tsFileResources) {
// TODO: try filtering files if the query contains time filter
if (!testResourceDevice(tsFileResource, deviceId)) {
continue;
}
closeQueryLock.readLock().lock();
try {
if (tsFileResource.isClosed()) {
tsfileResourcesForQuery.add(tsFileResource);
} else {
// left: in-memory data, right: meta of disk data
Pair<ReadOnlyMemChunk, List<ChunkMetaData>> pair = tsFileResource.getUnsealedFileProcessor()
.query(deviceId, measurementId, dataType, mSchema.getProps(), context);
tsfileResourcesForQuery.add(new TsFileResource(tsFileResource.getFile(),
tsFileResource.getStartTimeMap(), tsFileResource.getEndTimeMap(), pair.left, pair.right));
}
} finally {
closeQueryLock.readLock().unlock();
}
}
return tsfileResourcesForQuery;
}
/**
* @return true if the device is contained in the TsFile and it lives beyond TTL
*/
private boolean testResourceDevice(TsFileResource tsFileResource, String deviceId) {
if (!tsFileResource.containsDevice(deviceId)) {
return false;
}
if (dataTTL != Long.MAX_VALUE) {
Long deviceEndTime = tsFileResource.getEndTimeMap().get(deviceId);
return deviceEndTime == null || checkTTL(deviceEndTime);
}
return true;
}
/**
* Delete data whose timestamp <= 'timestamp' and belongs to the timeseries
* deviceId.measurementId.
*
* @param deviceId the deviceId of the timeseries to be deleted.
* @param measurementId the measurementId of the timeseries to be deleted.
* @param timestamp the delete range is (0, timestamp].
*/
public void delete(String deviceId, String measurementId, long timestamp) throws IOException {
// TODO: how to avoid partial deletion?
writeLock();
mergeLock.writeLock().lock();
// record files which are updated so that we can roll back them in case of exception
List<ModificationFile> updatedModFiles = new ArrayList<>();
try {
Long lastUpdateTime = latestTimeForEachDevice.get(deviceId);
// no tsfile data, the delete operation is invalid
if (lastUpdateTime == null) {
logger.debug("No device {} in SG {}, deletion invalid", deviceId, storageGroupName);
return;
}
// write log
if (IoTDBDescriptor.getInstance().getConfig().isEnableWal()) {
if (workSequenceTsFileProcessor != null) {
workSequenceTsFileProcessor.getLogNode()
.write(new DeletePlan(timestamp, new Path(deviceId, measurementId)));
}
if (workUnSequenceTsFileProcessor != null) {
workUnSequenceTsFileProcessor.getLogNode()
.write(new DeletePlan(timestamp, new Path(deviceId, measurementId)));
}
}
Path fullPath = new Path(deviceId, measurementId);
Deletion deletion = new Deletion(fullPath, versionController.nextVersion(), timestamp);
if (mergingModification != null) {
mergingModification.write(deletion);
updatedModFiles.add(mergingModification);
}
deleteDataInFiles(sequenceFileList, deletion, updatedModFiles);
deleteDataInFiles(unSequenceFileList, deletion, updatedModFiles);
} catch (Exception e) {
// roll back
for (ModificationFile modFile : updatedModFiles) {
modFile.abort();
}
throw new IOException(e);
} finally {
writeUnlock();
mergeLock.writeLock().unlock();
}
}
private void deleteDataInFiles(List<TsFileResource> tsFileResourceList, Deletion deletion,
List<ModificationFile> updatedModFiles)
throws IOException {
String deviceId = deletion.getDevice();
for (TsFileResource tsFileResource : tsFileResourceList) {
if (!tsFileResource.containsDevice(deviceId) ||
deletion.getTimestamp() < tsFileResource.getStartTimeMap().get(deviceId)) {
continue;
}
// write deletion into modification file
tsFileResource.getModFile().write(deletion);
// delete data in memory of unsealed file
if (!tsFileResource.isClosed()) {
TsFileProcessor tsfileProcessor = tsFileResource.getUnsealedFileProcessor();
tsfileProcessor.deleteDataInMemory(deletion);
}
// add a record in case of rollback
updatedModFiles.add(tsFileResource.getModFile());
}
}
/**
* when close an TsFileProcessor, update its EndTimeMap immediately
*
* @param tsFileProcessor processor to be closed
*/
private void updateEndTimeMap(TsFileProcessor tsFileProcessor) {
TsFileResource resource = tsFileProcessor.getTsFileResource();
for (Entry<String, Long> startTime : resource.getStartTimeMap().entrySet()) {
String deviceId = startTime.getKey();
resource.forceUpdateEndTime(deviceId, latestTimeForEachDevice.get(deviceId));
}
}
private boolean updateLatestFlushTimeCallback() {
// update the largest timestamp in the last flushing memtable
for (Entry<String, Long> entry : latestTimeForEachDevice.entrySet()) {
latestFlushedTimeForEachDevice.put(entry.getKey(), entry.getValue());
}
return true;
}
/**
* put the memtable back to the MemTablePool and make the metadata in writer visible
*/
// TODO please consider concurrency with query and insert method.
private void closeUnsealedTsFileProcessor(
TsFileProcessor tsFileProcessor) throws TsFileProcessorException {
closeQueryLock.writeLock().lock();
try {
tsFileProcessor.close();
} finally {
closeQueryLock.writeLock().unlock();
}
//closingSequenceTsFileProcessor is a thread safety class.
if (closingSequenceTsFileProcessor.contains(tsFileProcessor)) {
closingSequenceTsFileProcessor.remove(tsFileProcessor);
} else {
closingUnSequenceTsFileProcessor.remove(tsFileProcessor);
}
logger.info("signal closing storage group condition in {}", storageGroupName);
synchronized (closeStorageGroupCondition) {
closeStorageGroupCondition.notifyAll();
}
}
/**
* count all Tsfiles in the storage group which need to be upgraded
*
* @return total num of the tsfiles which need to be upgraded in the storage group
*/
public int countUpgradeFiles() {
int cntUpgradeFileNum = 0;
for (TsFileResource seqTsFileResource : sequenceFileList) {
if (UpgradeUtils.isNeedUpgrade(seqTsFileResource)) {
cntUpgradeFileNum += 1;
}
}
for (TsFileResource unseqTsFileResource : unSequenceFileList) {
if (UpgradeUtils.isNeedUpgrade(unseqTsFileResource)) {
cntUpgradeFileNum += 1;
}
}
return cntUpgradeFileNum;
}
public void upgrade() {
for (TsFileResource seqTsFileResource : sequenceFileList) {
seqTsFileResource.doUpgrade();
}
for (TsFileResource unseqTsFileResource : unSequenceFileList) {
unseqTsFileResource.doUpgrade();
}
}
public void merge(boolean fullMerge) {
writeLock();
try {
if (isMerging) {
if (logger.isInfoEnabled()) {
logger.info("{} Last merge is ongoing, currently consumed time: {}ms", storageGroupName,
(System.currentTimeMillis() - mergeStartTime));
}
return;
}
if (unSequenceFileList.isEmpty() || sequenceFileList.isEmpty()) {
logger.info("{} no files to be merged", storageGroupName);
return;
}
long budget = IoTDBDescriptor.getInstance().getConfig().getMergeMemoryBudget();
long timeLowerBound = System.currentTimeMillis() - dataTTL;
MergeResource mergeResource = new MergeResource(sequenceFileList, unSequenceFileList,
timeLowerBound);
IMergeFileSelector fileSelector = getMergeFileSelector(budget, mergeResource);
try {
List[] mergeFiles = fileSelector.select();
if (mergeFiles.length == 0) {