From 91c50e165625646e65620c93d9809ba100693c7a Mon Sep 17 00:00:00 2001 From: yyt86 Date: Tue, 6 Jul 2021 12:54:52 -0500 Subject: [PATCH 01/28] change parameter 'timeLowerBound' to 'expiredTime' in FileTimeIndex.java --- .../db/engine/storagegroup/timeindex/FileTimeIndex.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/timeindex/FileTimeIndex.java b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/timeindex/FileTimeIndex.java index e1e5251cc1f28..14791217ffbe1 100644 --- a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/timeindex/FileTimeIndex.java +++ b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/timeindex/FileTimeIndex.java @@ -119,12 +119,12 @@ public boolean endTimeEmpty() { } @Override - public boolean stillLives(long timeLowerBound) { - if (timeLowerBound == Long.MAX_VALUE) { + public boolean stillLives(long expiredTime) { + if (expiredTime == Long.MAX_VALUE) { return true; } // the file cannot be deleted if any device still lives - return endTime >= timeLowerBound; + return endTime >= expiredTime; } @Override From cbc8b3fc54e05dc6609eb17f3c83a90b11b45faf Mon Sep 17 00:00:00 2001 From: yyt86 Date: Tue, 6 Jul 2021 13:04:34 -0500 Subject: [PATCH 02/28] change the position of the 'getProcessor' method --- .../apache/iotdb/db/engine/storagegroup/TsFileResource.java | 4 +--- .../db/engine/storagegroup/timeindex/DeviceTimeIndex.java | 6 +++--- 2 files changed, 4 insertions(+), 6 deletions(-) diff --git a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileResource.java b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileResource.java index 1e1b63d4f88a2..d238428ad26bf 100644 --- a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileResource.java +++ b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileResource.java @@ -89,12 +89,10 @@ public class TsFileResource { /** version number */ public static final byte VERSION_NUMBER = 1; + private TsFileProcessor processor; public TsFileProcessor getProcessor() { return processor; } - - private TsFileProcessor processor; - /** time index */ protected ITimeIndex timeIndex; diff --git a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/timeindex/DeviceTimeIndex.java b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/timeindex/DeviceTimeIndex.java index 381d4b36a4a73..bfbfda7b5eaeb 100644 --- a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/timeindex/DeviceTimeIndex.java +++ b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/timeindex/DeviceTimeIndex.java @@ -164,13 +164,13 @@ public boolean endTimeEmpty() { } @Override - public boolean stillLives(long timeLowerBound) { - if (timeLowerBound == Long.MAX_VALUE) { + public boolean stillLives(long expiredTime) { + if (expiredTime == Long.MAX_VALUE) { return true; } for (long endTime : endTimes) { // the file cannot be deleted if any device still lives - if (endTime >= timeLowerBound) { + if (endTime >= expiredTime) { return true; } } From 61f8b08ecd0d5fe9fce376e52a708fbd5b0fb519 Mon Sep 17 00:00:00 2001 From: Yuting Date: Sat, 10 Jul 2021 19:32:34 -0500 Subject: [PATCH 03/28] modify TsFileResource --- .../db/engine/storagegroup/timeindex/DeviceTimeIndex.java | 6 +++--- .../db/engine/storagegroup/timeindex/FileTimeIndex.java | 6 +++--- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/timeindex/DeviceTimeIndex.java b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/timeindex/DeviceTimeIndex.java index bfbfda7b5eaeb..d709856ef001c 100644 --- a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/timeindex/DeviceTimeIndex.java +++ b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/timeindex/DeviceTimeIndex.java @@ -164,13 +164,13 @@ public boolean endTimeEmpty() { } @Override - public boolean stillLives(long expiredTime) { - if (expiredTime == Long.MAX_VALUE) { + public boolean stillLives(long ttlLowerBound) { + if (ttlLowerBound == Long.MAX_VALUE) { return true; } for (long endTime : endTimes) { // the file cannot be deleted if any device still lives - if (endTime >= expiredTime) { + if (endTime >= ttlLowerBound) { return true; } } diff --git a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/timeindex/FileTimeIndex.java b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/timeindex/FileTimeIndex.java index 14791217ffbe1..2bc4b3b935249 100644 --- a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/timeindex/FileTimeIndex.java +++ b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/timeindex/FileTimeIndex.java @@ -119,12 +119,12 @@ public boolean endTimeEmpty() { } @Override - public boolean stillLives(long expiredTime) { - if (expiredTime == Long.MAX_VALUE) { + public boolean stillLives(long ttlLowerBound) { + if (ttlLowerBound == Long.MAX_VALUE) { return true; } // the file cannot be deleted if any device still lives - return endTime >= expiredTime; + return endTime >= ttlLowerBound; } @Override From 09fb62b7ab1d8f6b81f0705616573b2f9d4996f9 Mon Sep 17 00:00:00 2001 From: Yuting Date: Mon, 12 Jul 2021 00:16:22 -0500 Subject: [PATCH 04/28] don't change timeLowerBound in TsFileResource --- .../engine/compaction/TsFileManagement.java | 4 ++-- .../db/engine/memtable/AbstractMemTable.java | 2 +- .../iotdb/db/engine/memtable/IMemTable.java | 2 +- .../db/engine/merge/manage/MergeResource.java | 8 ++++---- .../storagegroup/StorageGroupProcessor.java | 20 +++++++++---------- .../engine/storagegroup/TsFileResource.java | 1 + .../storagegroup/timeindex/ITimeIndex.java | 4 ++-- .../merge/MaxFileMergeFileSelectorTest.java | 5 ++--- 8 files changed, 23 insertions(+), 23 deletions(-) diff --git a/server/src/main/java/org/apache/iotdb/db/engine/compaction/TsFileManagement.java b/server/src/main/java/org/apache/iotdb/db/engine/compaction/TsFileManagement.java index 8f17c1f776eed..e49d198abe417 100644 --- a/server/src/main/java/org/apache/iotdb/db/engine/compaction/TsFileManagement.java +++ b/server/src/main/java/org/apache/iotdb/db/engine/compaction/TsFileManagement.java @@ -241,8 +241,8 @@ public synchronized void merge( } long budget = IoTDBDescriptor.getInstance().getConfig().getMergeMemoryBudget(); - long timeLowerBound = System.currentTimeMillis() - dataTTL; - MergeResource mergeResource = new MergeResource(seqMergeList, unSeqMergeList, timeLowerBound); + long ttlLowerBound = System.currentTimeMillis() - dataTTL; + MergeResource mergeResource = new MergeResource(seqMergeList, unSeqMergeList, ttlLowerBound); IMergeFileSelector fileSelector = getMergeFileSelector(budget, mergeResource); try { diff --git a/server/src/main/java/org/apache/iotdb/db/engine/memtable/AbstractMemTable.java b/server/src/main/java/org/apache/iotdb/db/engine/memtable/AbstractMemTable.java index 170f49cfe5e37..0acb9e11e7f08 100644 --- a/server/src/main/java/org/apache/iotdb/db/engine/memtable/AbstractMemTable.java +++ b/server/src/main/java/org/apache/iotdb/db/engine/memtable/AbstractMemTable.java @@ -306,7 +306,7 @@ public ReadOnlyMemChunk query( String deviceId, String measurement, IMeasurementSchema partialVectorSchema, - long timeLowerBound, + long ttlLowerBound, List deletionList) throws IOException, QueryProcessException { if (partialVectorSchema.getType() == TSDataType.VECTOR) { diff --git a/server/src/main/java/org/apache/iotdb/db/engine/memtable/IMemTable.java b/server/src/main/java/org/apache/iotdb/db/engine/memtable/IMemTable.java index bb28c57530c9e..e5c9e32a9c7d0 100644 --- a/server/src/main/java/org/apache/iotdb/db/engine/memtable/IMemTable.java +++ b/server/src/main/java/org/apache/iotdb/db/engine/memtable/IMemTable.java @@ -92,7 +92,7 @@ ReadOnlyMemChunk query( String deviceId, String measurement, IMeasurementSchema schema, - long timeLowerBound, + long ttlLowerBound, List deletionList) throws IOException, QueryProcessException, MetadataException; diff --git a/server/src/main/java/org/apache/iotdb/db/engine/merge/manage/MergeResource.java b/server/src/main/java/org/apache/iotdb/db/engine/merge/manage/MergeResource.java index d7bc827f538b0..bde31e257cc5f 100644 --- a/server/src/main/java/org/apache/iotdb/db/engine/merge/manage/MergeResource.java +++ b/server/src/main/java/org/apache/iotdb/db/engine/merge/manage/MergeResource.java @@ -68,7 +68,7 @@ public class MergeResource { new HashMap<>(); // is this too waste? private Map chunkWriterCache = new ConcurrentHashMap<>(); - private long timeLowerBound = Long.MIN_VALUE; + private long ttlLowerBound = Long.MIN_VALUE; private boolean cacheDeviceMeta = false; @@ -81,12 +81,12 @@ public MergeResource(List seqFiles, List unseqFi private boolean filterResource(TsFileResource res) { return res.getTsFile().exists() && !res.isDeleted() - && (!res.isClosed() || res.stillLives(timeLowerBound)); + && (!res.isClosed() || res.stillLives(ttlLowerBound)); } public MergeResource( - Collection seqFiles, List unseqFiles, long timeLowerBound) { - this.timeLowerBound = timeLowerBound; + Collection seqFiles, List unseqFiles, long ttlLowerBound) { + this.ttlLowerBound = ttlLowerBound; this.seqFiles = seqFiles.stream().filter(this::filterResource).collect(Collectors.toList()); this.unseqFiles = unseqFiles.stream().filter(this::filterResource).collect(Collectors.toList()); } diff --git a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/StorageGroupProcessor.java b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/StorageGroupProcessor.java index bb3820ef10998..5439f65bcbf74 100755 --- a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/StorageGroupProcessor.java +++ b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/StorageGroupProcessor.java @@ -754,7 +754,7 @@ private void recoverTsFiles(List tsFiles, boolean isSeq) { virtualStorageGroupId, storageGroupInfo, tsFileResource, - this::closeUnsealedTsFileProcessorCallBack, + this::closeUnsealedTsFileProcessorCallBack, // ????? delete this::updateLatestFlushTimeCallback, true, writer); @@ -1487,12 +1487,12 @@ public synchronized void checkFilesTTL() { logicalStorageGroupName + "-" + virtualStorageGroupId); return; } - long timeLowerBound = System.currentTimeMillis() - dataTTL; + long ttlLowerBound = System.currentTimeMillis() - dataTTL; if (logger.isDebugEnabled()) { logger.debug( "{}: TTL removing files before {}", logicalStorageGroupName + "-" + virtualStorageGroupId, - new Date(timeLowerBound)); + new Date(ttlLowerBound)); } // copy to avoid concurrent modification of deletion @@ -1500,17 +1500,17 @@ public synchronized void checkFilesTTL() { List unseqFiles = new ArrayList<>(tsFileManagement.getTsFileList(false)); for (TsFileResource tsFileResource : seqFiles) { - checkFileTTL(tsFileResource, timeLowerBound, true); + checkFileTTL(tsFileResource, ttlLowerBound, true); } for (TsFileResource tsFileResource : unseqFiles) { - checkFileTTL(tsFileResource, timeLowerBound, false); + checkFileTTL(tsFileResource, ttlLowerBound, false); } } - private void checkFileTTL(TsFileResource resource, long timeLowerBound, boolean isSeq) { + private void checkFileTTL(TsFileResource resource, long ttlLowerBound, boolean isSeq) { if (resource.isMerging() || !resource.isClosed() - || !resource.isDeleted() && resource.stillLives(timeLowerBound)) { + || !resource.isDeleted() && resource.stillLives(ttlLowerBound)) { return; } @@ -1533,7 +1533,7 @@ private void checkFileTTL(TsFileResource resource, long timeLowerBound, boolean logger.info( "Removed a file {} before {} by ttl ({}ms)", resource.getTsFilePath(), - new Date(timeLowerBound), + new Date(ttlLowerBound), dataTTL); } tsFileManagement.remove(resource, isSeq); @@ -1719,9 +1719,9 @@ private List getFileResourceListForQuery( IMeasurementSchema schema = IoTDB.metaManager.getSeriesSchema(fullPath); List tsfileResourcesForQuery = new ArrayList<>(); - long timeLowerBound = + long ttlLowerBound = dataTTL != Long.MAX_VALUE ? System.currentTimeMillis() - dataTTL : Long.MIN_VALUE; - context.setQueryTimeLowerBound(timeLowerBound); + context.setQueryTimeLowerBound(ttlLowerBound); // for upgrade files and old files must be closed for (TsFileResource tsFileResource : upgradeTsFileResources) { diff --git a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileResource.java b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileResource.java index d238428ad26bf..8f7dc815f3e26 100644 --- a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileResource.java +++ b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileResource.java @@ -90,6 +90,7 @@ public class TsFileResource { public static final byte VERSION_NUMBER = 1; private TsFileProcessor processor; + public TsFileProcessor getProcessor() { return processor; } diff --git a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/timeindex/ITimeIndex.java b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/timeindex/ITimeIndex.java index 4c87408484828..ba98c3429cc47 100644 --- a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/timeindex/ITimeIndex.java +++ b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/timeindex/ITimeIndex.java @@ -68,10 +68,10 @@ public interface ITimeIndex { boolean endTimeEmpty(); /** - * @param timeLowerBound time lower bound + * @param ttlLowerBound time lower bound * @return whether any of the device lives over the given time bound */ - boolean stillLives(long timeLowerBound); + boolean stillLives(long ttlLowerBound); /** @return Calculate file index ram size */ long calculateRamSize(); diff --git a/server/src/test/java/org/apache/iotdb/db/engine/merge/MaxFileMergeFileSelectorTest.java b/server/src/test/java/org/apache/iotdb/db/engine/merge/MaxFileMergeFileSelectorTest.java index 789d58dabe6b6..f852a88122309 100644 --- a/server/src/test/java/org/apache/iotdb/db/engine/merge/MaxFileMergeFileSelectorTest.java +++ b/server/src/test/java/org/apache/iotdb/db/engine/merge/MaxFileMergeFileSelectorTest.java @@ -192,9 +192,8 @@ public void testFileOpenSelectionFromCompaction() List newUnseqResources = new ArrayList<>(); newUnseqResources.add(largeUnseqTsFileResource); - long timeLowerBound = System.currentTimeMillis() - Long.MAX_VALUE; - MergeResource mergeResource = - new MergeResource(seqResources, newUnseqResources, timeLowerBound); + long ttlLowerBound = System.currentTimeMillis() - Long.MAX_VALUE; + MergeResource mergeResource = new MergeResource(seqResources, newUnseqResources, ttlLowerBound); assertEquals(5, mergeResource.getSeqFiles().size()); assertEquals(1, mergeResource.getUnseqFiles().size()); mergeResource.clear(); From 2b991f3ca645903fc773ebb40da1e6fb4b763fdf Mon Sep 17 00:00:00 2001 From: Yuting Date: Mon, 12 Jul 2021 08:40:53 -0500 Subject: [PATCH 05/28] remove comment --- .../iotdb/db/engine/storagegroup/StorageGroupProcessor.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/StorageGroupProcessor.java b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/StorageGroupProcessor.java index 5439f65bcbf74..66110a17822eb 100755 --- a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/StorageGroupProcessor.java +++ b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/StorageGroupProcessor.java @@ -754,7 +754,7 @@ private void recoverTsFiles(List tsFiles, boolean isSeq) { virtualStorageGroupId, storageGroupInfo, tsFileResource, - this::closeUnsealedTsFileProcessorCallBack, // ????? delete + this::closeUnsealedTsFileProcessorCallBack, this::updateLatestFlushTimeCallback, true, writer); From 93adc8d20dc4112bba26b08808646761f5421aee Mon Sep 17 00:00:00 2001 From: Yuting Date: Tue, 13 Jul 2021 00:36:52 -0500 Subject: [PATCH 06/28] remove deviceNumInLastClosedTsFile --- .../engine/storagegroup/StorageGroupProcessor.java | 7 ++----- .../db/engine/storagegroup/TsFileProcessor.java | 5 ++--- .../db/engine/storagegroup/TsFileResource.java | 4 ++-- .../storagegroup/timeindex/DeviceTimeIndex.java | 8 -------- .../storagegroup/timeindex/TimeIndexLevel.java | 10 ---------- .../engine/storagegroup/TsFileProcessorTest.java | 14 ++++---------- 6 files changed, 10 insertions(+), 38 deletions(-) diff --git a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/StorageGroupProcessor.java b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/StorageGroupProcessor.java index 66110a17822eb..6932095fe44c4 100755 --- a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/StorageGroupProcessor.java +++ b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/StorageGroupProcessor.java @@ -1263,8 +1263,7 @@ private TsFileProcessor getTsFileProcessor( storageGroupInfo, this::closeUnsealedTsFileProcessorCallBack, this::updateLatestFlushTimeCallback, - true, - deviceNumInLastClosedTsFile); + true); } else { tsFileProcessor = new TsFileProcessor( @@ -1273,8 +1272,7 @@ private TsFileProcessor getTsFileProcessor( storageGroupInfo, this::closeUnsealedTsFileProcessorCallBack, this::unsequenceFlushCallback, - false, - deviceNumInLastClosedTsFile); + false); } if (enableMemControl) { @@ -2024,7 +2022,6 @@ private void closeUnsealedTsFileProcessorCallBack(TsFileProcessor tsFileProcesso closeQueryLock.writeLock().lock(); try { tsFileProcessor.close(); - deviceNumInLastClosedTsFile = tsFileProcessor.getTsFileResource().getDevices().size(); } finally { closeQueryLock.writeLock().unlock(); } diff --git a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileProcessor.java b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileProcessor.java index 28f0f6e49d193..ff19c8991b1ab 100644 --- a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileProcessor.java +++ b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileProcessor.java @@ -160,11 +160,10 @@ public class TsFileProcessor { StorageGroupInfo storageGroupInfo, CloseFileListener closeTsFileCallback, UpdateEndTimeCallBack updateLatestFlushTimeCallback, - boolean sequence, - int deviceNumInLastClosedTsFile) + boolean sequence) throws IOException { this.storageGroupName = storageGroupName; - this.tsFileResource = new TsFileResource(tsfile, this, deviceNumInLastClosedTsFile); + this.tsFileResource = new TsFileResource(tsfile, this); this.storageGroupInfo = storageGroupInfo; this.writer = new RestorableTsFileIOWriter(tsfile); this.updateLatestFlushTimeCallback = updateLatestFlushTimeCallback; diff --git a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileResource.java b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileResource.java index 8f7dc815f3e26..9a8bc81f655a5 100644 --- a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileResource.java +++ b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileResource.java @@ -184,10 +184,10 @@ public TsFileResource(File file) { } /** unsealed TsFile */ - public TsFileResource(File file, TsFileProcessor processor, int deviceNumInLastClosedTsFile) { + public TsFileResource(File file, TsFileProcessor processor) { this.file = file; this.version = FilePathUtils.splitAndGetTsFileVersion(this.file.getName()); - this.timeIndex = config.getTimeIndexLevel().getTimeIndex(deviceNumInLastClosedTsFile); + this.timeIndex = config.getTimeIndexLevel().getTimeIndex(); this.timeIndexType = (byte) config.getTimeIndexLevel().ordinal(); this.processor = processor; } diff --git a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/timeindex/DeviceTimeIndex.java b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/timeindex/DeviceTimeIndex.java index c0556a9fc0f36..ec8d47f468893 100644 --- a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/timeindex/DeviceTimeIndex.java +++ b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/timeindex/DeviceTimeIndex.java @@ -64,14 +64,6 @@ public DeviceTimeIndex() { initTimes(endTimes, Long.MIN_VALUE); } - public DeviceTimeIndex(int deviceNumInLastClosedTsFile) { - this.deviceToIndex = new ConcurrentHashMap<>(); - this.startTimes = new long[deviceNumInLastClosedTsFile]; - this.endTimes = new long[deviceNumInLastClosedTsFile]; - initTimes(startTimes, Long.MAX_VALUE); - initTimes(endTimes, Long.MIN_VALUE); - } - public DeviceTimeIndex(Map deviceToIndex, long[] startTimes, long[] endTimes) { this.startTimes = startTimes; this.endTimes = endTimes; diff --git a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/timeindex/TimeIndexLevel.java b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/timeindex/TimeIndexLevel.java index 93650fbfe9d23..95e9b9eee8fc5 100644 --- a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/timeindex/TimeIndexLevel.java +++ b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/timeindex/TimeIndexLevel.java @@ -36,16 +36,6 @@ public ITimeIndex getTimeIndex() { } } - public ITimeIndex getTimeIndex(int deviceNumInLastClosedTsFile) { - switch (this) { - case FILE_TIME_INDEX: - return new FileTimeIndex(); - case DEVICE_TIME_INDEX: - default: - return new DeviceTimeIndex(deviceNumInLastClosedTsFile); - } - } - public static TimeIndexLevel valueOf(int ordinal) { if (ordinal < 0 || ordinal >= values().length) { throw new IndexOutOfBoundsException("Invalid ordinal"); diff --git a/server/src/test/java/org/apache/iotdb/db/engine/storagegroup/TsFileProcessorTest.java b/server/src/test/java/org/apache/iotdb/db/engine/storagegroup/TsFileProcessorTest.java index ad963833ad3a2..e72ee80c5e2d1 100644 --- a/server/src/test/java/org/apache/iotdb/db/engine/storagegroup/TsFileProcessorTest.java +++ b/server/src/test/java/org/apache/iotdb/db/engine/storagegroup/TsFileProcessorTest.java @@ -71,8 +71,6 @@ public class TsFileProcessorTest { private QueryContext context; private static Logger logger = LoggerFactory.getLogger(TsFileProcessorTest.class); - protected static final int INIT_ARRAY_SIZE = 64; - @Before public void setUp() { EnvironmentUtils.envSetUp(); @@ -96,8 +94,7 @@ public void testWriteAndFlush() throws IOException, WriteProcessException, Metad sgInfo, this::closeTsFileProcessor, (tsFileProcessor) -> true, - true, - INIT_ARRAY_SIZE); + true); TsFileProcessorInfo tsFileProcessorInfo = new TsFileProcessorInfo(sgInfo); processor.setTsFileProcessorInfo(tsFileProcessorInfo); @@ -172,8 +169,7 @@ public void testWriteAndRestoreMetadata() sgInfo, this::closeTsFileProcessor, (tsFileProcessor) -> true, - true, - INIT_ARRAY_SIZE); + true); TsFileProcessorInfo tsFileProcessorInfo = new TsFileProcessorInfo(sgInfo); processor.setTsFileProcessorInfo(tsFileProcessorInfo); @@ -274,8 +270,7 @@ public void testMultiFlush() throws IOException, WriteProcessException, Metadata sgInfo, this::closeTsFileProcessor, (tsFileProcessor) -> true, - true, - INIT_ARRAY_SIZE); + true); TsFileProcessorInfo tsFileProcessorInfo = new TsFileProcessorInfo(sgInfo); processor.setTsFileProcessorInfo(tsFileProcessorInfo); @@ -330,8 +325,7 @@ public void testWriteAndClose() throws IOException, WriteProcessException, Metad sgInfo, this::closeTsFileProcessor, (tsFileProcessor) -> true, - true, - INIT_ARRAY_SIZE); + true); TsFileProcessorInfo tsFileProcessorInfo = new TsFileProcessorInfo(sgInfo); processor.setTsFileProcessorInfo(tsFileProcessorInfo); From f767d62623a330d0f3ace2931436937a1eee76e9 Mon Sep 17 00:00:00 2001 From: Yuting Date: Tue, 13 Jul 2021 22:02:19 -0500 Subject: [PATCH 07/28] add checkDeviceIdExist in ITimeIndex --- .../db/engine/storagegroup/TsFileResource.java | 2 +- .../storagegroup/timeindex/DeviceTimeIndex.java | 14 ++++++++++++++ .../storagegroup/timeindex/FileTimeIndex.java | 5 +++++ .../engine/storagegroup/timeindex/ITimeIndex.java | 8 ++++++++ .../apache/iotdb/db/qp/utils/DatetimeUtils.java | 4 ++++ 5 files changed, 32 insertions(+), 1 deletion(-) diff --git a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileResource.java b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileResource.java index 9a8bc81f655a5..20ffba46faadb 100644 --- a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileResource.java +++ b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileResource.java @@ -630,7 +630,7 @@ public boolean stillLives(long timeLowerBound) { /** @return true if the device is contained in the TsFile and it lives beyond TTL */ public boolean isSatisfied( String deviceId, Filter timeFilter, boolean isSeq, long ttl, boolean debug) { - if (!getDevices().contains(deviceId)) { + if (!timeIndex.checkDeviceIdExist(deviceId)) { if (debug) { DEBUG_LOGGER.info( "Path: {} file {} is not satisfied because of no device!", deviceId, file); diff --git a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/timeindex/DeviceTimeIndex.java b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/timeindex/DeviceTimeIndex.java index ec8d47f468893..fe8951fd3141e 100644 --- a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/timeindex/DeviceTimeIndex.java +++ b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/timeindex/DeviceTimeIndex.java @@ -27,6 +27,9 @@ import org.apache.iotdb.tsfile.utils.RamUsageEstimator; import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; @@ -44,6 +47,8 @@ public class DeviceTimeIndex implements ITimeIndex { protected static final Map cachedDevicePool = CachedStringPool.getInstance().getCachedPool(); + private static final Logger DEBUG_LOGGER = LoggerFactory.getLogger("QUERY_DEBUG"); + /** start times array. */ protected long[] startTimes; @@ -297,4 +302,13 @@ public long getEndTime(String deviceId) { } return endTimes[deviceToIndex.get(deviceId)]; } + + @Override + public boolean checkDeviceIdExist(String deviceId) { + if (!deviceToIndex.containsKey(deviceId)) { + return false; + } else { + return true; + } + } } diff --git a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/timeindex/FileTimeIndex.java b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/timeindex/FileTimeIndex.java index d8baafe93118d..d44d7362904a2 100644 --- a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/timeindex/FileTimeIndex.java +++ b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/timeindex/FileTimeIndex.java @@ -209,4 +209,9 @@ public long getStartTime(String deviceId) { public long getEndTime(String deviceId) { return endTime; } + + @Override + public boolean checkDeviceIdExist(String deviceId) { + return true; + } } diff --git a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/timeindex/ITimeIndex.java b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/timeindex/ITimeIndex.java index ba98c3429cc47..0d51be74e8011 100644 --- a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/timeindex/ITimeIndex.java +++ b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/timeindex/ITimeIndex.java @@ -147,4 +147,12 @@ public interface ITimeIndex { * @return end time */ long getEndTime(String deviceId); + + /** + * check whether deviceId exists in device set + * + * @param deviceId device name + * @return true if the deviceId exists in device set, otherwise false. + */ + boolean checkDeviceIdExist(String deviceId); } diff --git a/server/src/main/java/org/apache/iotdb/db/qp/utils/DatetimeUtils.java b/server/src/main/java/org/apache/iotdb/db/qp/utils/DatetimeUtils.java index 2ca9939663e6b..f449cea7dfdd1 100644 --- a/server/src/main/java/org/apache/iotdb/db/qp/utils/DatetimeUtils.java +++ b/server/src/main/java/org/apache/iotdb/db/qp/utils/DatetimeUtils.java @@ -33,6 +33,7 @@ import java.time.format.SignStyle; import java.time.temporal.ChronoField; import java.util.Calendar; +import java.util.TimeZone; import java.util.concurrent.TimeUnit; public class DatetimeUtils { @@ -583,6 +584,9 @@ public static long convertDurationStrToLong( res *= 30 * 86_400_000L; } else { Calendar calendar = Calendar.getInstance(); + TimeZone tz = TimeZone.getTimeZone("GMT+8"); + // calendar.setTimeZone(SessionManager.getInstance().getCurrSessionTimeZone()); + calendar.setTimeZone(tz); calendar.setTimeInMillis(currentTime); calendar.add(Calendar.MONTH, (int) (value)); res = calendar.getTimeInMillis() - currentTime; From 7fd57b176c6c8cad09519178ec2c195deb68bd84 Mon Sep 17 00:00:00 2001 From: Yuting Date: Tue, 13 Jul 2021 22:57:22 -0500 Subject: [PATCH 08/28] modify checkDeviceIdExist method --- .../db/engine/storagegroup/timeindex/DeviceTimeIndex.java | 6 +----- .../iotdb/db/engine/storagegroup/timeindex/ITimeIndex.java | 4 ++-- 2 files changed, 3 insertions(+), 7 deletions(-) diff --git a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/timeindex/DeviceTimeIndex.java b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/timeindex/DeviceTimeIndex.java index fe8951fd3141e..44df7fe9ea7d0 100644 --- a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/timeindex/DeviceTimeIndex.java +++ b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/timeindex/DeviceTimeIndex.java @@ -305,10 +305,6 @@ public long getEndTime(String deviceId) { @Override public boolean checkDeviceIdExist(String deviceId) { - if (!deviceToIndex.containsKey(deviceId)) { - return false; - } else { - return true; - } + return deviceToIndex.containsKey(deviceId); } } diff --git a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/timeindex/ITimeIndex.java b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/timeindex/ITimeIndex.java index 0d51be74e8011..37103629b2f8a 100644 --- a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/timeindex/ITimeIndex.java +++ b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/timeindex/ITimeIndex.java @@ -149,10 +149,10 @@ public interface ITimeIndex { long getEndTime(String deviceId); /** - * check whether deviceId exists in device set + * check whether deviceId exists in TsFile * * @param deviceId device name - * @return true if the deviceId exists in device set, otherwise false. + * @return true if the deviceId may exist in TsFile, otherwise false. */ boolean checkDeviceIdExist(String deviceId); } From aac14193350c0786dea7b0731448edfe969cdd80 Mon Sep 17 00:00:00 2001 From: Yuting Date: Tue, 13 Jul 2021 23:05:19 -0500 Subject: [PATCH 09/28] remove set timezone as GMT+8 --- .../iotdb/db/qp/utils/DatetimeUtils.java | 490 +++++++++--------- 1 file changed, 244 insertions(+), 246 deletions(-) diff --git a/server/src/main/java/org/apache/iotdb/db/qp/utils/DatetimeUtils.java b/server/src/main/java/org/apache/iotdb/db/qp/utils/DatetimeUtils.java index f449cea7dfdd1..1ee5477badee0 100644 --- a/server/src/main/java/org/apache/iotdb/db/qp/utils/DatetimeUtils.java +++ b/server/src/main/java/org/apache/iotdb/db/qp/utils/DatetimeUtils.java @@ -20,6 +20,7 @@ import org.apache.iotdb.db.conf.IoTDBDescriptor; import org.apache.iotdb.db.exception.query.LogicalOperatorException; +import org.apache.iotdb.db.query.control.SessionManager; import org.apache.iotdb.db.utils.TestOnly; import java.time.DateTimeException; @@ -33,7 +34,6 @@ import java.time.format.SignStyle; import java.time.temporal.ChronoField; import java.util.Calendar; -import java.util.TimeZone; import java.util.concurrent.TimeUnit; public class DatetimeUtils { @@ -46,13 +46,13 @@ private DatetimeUtils() { static { ISO_LOCAL_DATE_WIDTH_1_2 = - new DateTimeFormatterBuilder() - .appendValue(ChronoField.YEAR, 4, 10, SignStyle.EXCEEDS_PAD) - .appendLiteral('-') - .appendValue(ChronoField.MONTH_OF_YEAR, 1, 2, SignStyle.NEVER) - .appendLiteral('-') - .appendValue(ChronoField.DAY_OF_MONTH, 1, 2, SignStyle.NEVER) - .toFormatter(); + new DateTimeFormatterBuilder() + .appendValue(ChronoField.YEAR, 4, 10, SignStyle.EXCEEDS_PAD) + .appendLiteral('-') + .appendValue(ChronoField.MONTH_OF_YEAR, 1, 2, SignStyle.NEVER) + .appendLiteral('-') + .appendValue(ChronoField.DAY_OF_MONTH, 1, 2, SignStyle.NEVER) + .toFormatter(); } /** such as '2011/12/03'. */ @@ -60,13 +60,13 @@ private DatetimeUtils() { static { ISO_LOCAL_DATE_WITH_SLASH = - new DateTimeFormatterBuilder() - .appendValue(ChronoField.YEAR, 4, 10, SignStyle.EXCEEDS_PAD) - .appendLiteral('/') - .appendValue(ChronoField.MONTH_OF_YEAR, 1, 2, SignStyle.NEVER) - .appendLiteral('/') - .appendValue(ChronoField.DAY_OF_MONTH, 1, 2, SignStyle.NEVER) - .toFormatter(); + new DateTimeFormatterBuilder() + .appendValue(ChronoField.YEAR, 4, 10, SignStyle.EXCEEDS_PAD) + .appendLiteral('/') + .appendValue(ChronoField.MONTH_OF_YEAR, 1, 2, SignStyle.NEVER) + .appendLiteral('/') + .appendValue(ChronoField.DAY_OF_MONTH, 1, 2, SignStyle.NEVER) + .toFormatter(); } /** such as '2011.12.03'. */ @@ -74,13 +74,13 @@ private DatetimeUtils() { static { ISO_LOCAL_DATE_WITH_DOT = - new DateTimeFormatterBuilder() - .appendValue(ChronoField.YEAR, 4, 10, SignStyle.EXCEEDS_PAD) - .appendLiteral('.') - .appendValue(ChronoField.MONTH_OF_YEAR, 1, 2, SignStyle.NEVER) - .appendLiteral('.') - .appendValue(ChronoField.DAY_OF_MONTH, 1, 2, SignStyle.NEVER) - .toFormatter(); + new DateTimeFormatterBuilder() + .appendValue(ChronoField.YEAR, 4, 10, SignStyle.EXCEEDS_PAD) + .appendLiteral('.') + .appendValue(ChronoField.MONTH_OF_YEAR, 1, 2, SignStyle.NEVER) + .appendLiteral('.') + .appendValue(ChronoField.DAY_OF_MONTH, 1, 2, SignStyle.NEVER) + .toFormatter(); } /** such as '10:15:30' or '10:15:30.123'. */ @@ -88,17 +88,17 @@ private DatetimeUtils() { static { ISO_LOCAL_TIME_WITH_MS = - new DateTimeFormatterBuilder() - .appendValue(ChronoField.HOUR_OF_DAY, 2) - .appendLiteral(':') - .appendValue(ChronoField.MINUTE_OF_HOUR, 2) - .appendLiteral(':') - .appendValue(ChronoField.SECOND_OF_MINUTE, 2) - .optionalStart() - .appendLiteral('.') - .appendValue(ChronoField.MILLI_OF_SECOND, 3) - .optionalEnd() - .toFormatter(); + new DateTimeFormatterBuilder() + .appendValue(ChronoField.HOUR_OF_DAY, 2) + .appendLiteral(':') + .appendValue(ChronoField.MINUTE_OF_HOUR, 2) + .appendLiteral(':') + .appendValue(ChronoField.SECOND_OF_MINUTE, 2) + .optionalStart() + .appendLiteral('.') + .appendValue(ChronoField.MILLI_OF_SECOND, 3) + .optionalEnd() + .toFormatter(); } /** such as '10:15:30' or '10:15:30.123456'. */ @@ -106,17 +106,17 @@ private DatetimeUtils() { static { ISO_LOCAL_TIME_WITH_US = - new DateTimeFormatterBuilder() - .appendValue(ChronoField.HOUR_OF_DAY, 2) - .appendLiteral(':') - .appendValue(ChronoField.MINUTE_OF_HOUR, 2) - .appendLiteral(':') - .appendValue(ChronoField.SECOND_OF_MINUTE, 2) - .optionalStart() - .appendLiteral('.') - .appendValue(ChronoField.MICRO_OF_SECOND, 6) - .optionalEnd() - .toFormatter(); + new DateTimeFormatterBuilder() + .appendValue(ChronoField.HOUR_OF_DAY, 2) + .appendLiteral(':') + .appendValue(ChronoField.MINUTE_OF_HOUR, 2) + .appendLiteral(':') + .appendValue(ChronoField.SECOND_OF_MINUTE, 2) + .optionalStart() + .appendLiteral('.') + .appendValue(ChronoField.MICRO_OF_SECOND, 6) + .optionalEnd() + .toFormatter(); } /** such as '10:15:30' or '10:15:30.123456789'. */ @@ -124,17 +124,17 @@ private DatetimeUtils() { static { ISO_LOCAL_TIME_WITH_NS = - new DateTimeFormatterBuilder() - .appendValue(ChronoField.HOUR_OF_DAY, 2) - .appendLiteral(':') - .appendValue(ChronoField.MINUTE_OF_HOUR, 2) - .appendLiteral(':') - .appendValue(ChronoField.SECOND_OF_MINUTE, 2) - .optionalStart() - .appendLiteral('.') - .appendValue(ChronoField.NANO_OF_SECOND, 9) - .optionalEnd() - .toFormatter(); + new DateTimeFormatterBuilder() + .appendValue(ChronoField.HOUR_OF_DAY, 2) + .appendLiteral(':') + .appendValue(ChronoField.MINUTE_OF_HOUR, 2) + .appendLiteral(':') + .appendValue(ChronoField.SECOND_OF_MINUTE, 2) + .optionalStart() + .appendLiteral('.') + .appendValue(ChronoField.NANO_OF_SECOND, 9) + .optionalEnd() + .toFormatter(); } /** such as '2011-12-03T10:15:30+01:00' or '2011-12-03T10:15:30.123+01:00'. */ @@ -142,13 +142,13 @@ private DatetimeUtils() { static { ISO_OFFSET_DATE_TIME_WITH_MS = - new DateTimeFormatterBuilder() - .parseCaseInsensitive() - .append(ISO_LOCAL_DATE_WIDTH_1_2) - .appendLiteral('T') - .append(ISO_LOCAL_TIME_WITH_MS) - .appendOffsetId() - .toFormatter(); + new DateTimeFormatterBuilder() + .parseCaseInsensitive() + .append(ISO_LOCAL_DATE_WIDTH_1_2) + .appendLiteral('T') + .append(ISO_LOCAL_TIME_WITH_MS) + .appendOffsetId() + .toFormatter(); } /** such as '2011-12-03T10:15:30+01:00' or '2011-12-03T10:15:30.123456+01:00'. */ @@ -156,13 +156,13 @@ private DatetimeUtils() { static { ISO_OFFSET_DATE_TIME_WITH_US = - new DateTimeFormatterBuilder() - .parseCaseInsensitive() - .append(ISO_LOCAL_DATE_WIDTH_1_2) - .appendLiteral('T') - .append(ISO_LOCAL_TIME_WITH_US) - .appendOffsetId() - .toFormatter(); + new DateTimeFormatterBuilder() + .parseCaseInsensitive() + .append(ISO_LOCAL_DATE_WIDTH_1_2) + .appendLiteral('T') + .append(ISO_LOCAL_TIME_WITH_US) + .appendOffsetId() + .toFormatter(); } /** such as '2011-12-03T10:15:30+01:00' or '2011-12-03T10:15:30.123456789+01:00'. */ @@ -170,13 +170,13 @@ private DatetimeUtils() { static { ISO_OFFSET_DATE_TIME_WITH_NS = - new DateTimeFormatterBuilder() - .parseCaseInsensitive() - .append(ISO_LOCAL_DATE_WIDTH_1_2) - .appendLiteral('T') - .append(ISO_LOCAL_TIME_WITH_NS) - .appendOffsetId() - .toFormatter(); + new DateTimeFormatterBuilder() + .parseCaseInsensitive() + .append(ISO_LOCAL_DATE_WIDTH_1_2) + .appendLiteral('T') + .append(ISO_LOCAL_TIME_WITH_NS) + .appendOffsetId() + .toFormatter(); } /** such as '2011/12/03T10:15:30+01:00' or '2011/12/03T10:15:30.123+01:00'. */ @@ -184,13 +184,13 @@ private DatetimeUtils() { static { ISO_OFFSET_DATE_TIME_WITH_SLASH = - new DateTimeFormatterBuilder() - .parseCaseInsensitive() - .append(ISO_LOCAL_DATE_WITH_SLASH) - .appendLiteral('T') - .append(ISO_LOCAL_TIME_WITH_MS) - .appendOffsetId() - .toFormatter(); + new DateTimeFormatterBuilder() + .parseCaseInsensitive() + .append(ISO_LOCAL_DATE_WITH_SLASH) + .appendLiteral('T') + .append(ISO_LOCAL_TIME_WITH_MS) + .appendOffsetId() + .toFormatter(); } /** such as '2011/12/03T10:15:30+01:00' or '2011/12/03T10:15:30.123456+01:00'. */ @@ -198,13 +198,13 @@ private DatetimeUtils() { static { ISO_OFFSET_DATE_TIME_WITH_SLASH_US = - new DateTimeFormatterBuilder() - .parseCaseInsensitive() - .append(ISO_LOCAL_DATE_WITH_SLASH) - .appendLiteral('T') - .append(ISO_LOCAL_TIME_WITH_US) - .appendOffsetId() - .toFormatter(); + new DateTimeFormatterBuilder() + .parseCaseInsensitive() + .append(ISO_LOCAL_DATE_WITH_SLASH) + .appendLiteral('T') + .append(ISO_LOCAL_TIME_WITH_US) + .appendOffsetId() + .toFormatter(); } /** such as '2011/12/03T10:15:30+01:00' or '2011/12/03T10:15:30.123456789+01:00'. */ @@ -212,13 +212,13 @@ private DatetimeUtils() { static { ISO_OFFSET_DATE_TIME_WITH_SLASH_NS = - new DateTimeFormatterBuilder() - .parseCaseInsensitive() - .append(ISO_LOCAL_DATE_WITH_SLASH) - .appendLiteral('T') - .append(ISO_LOCAL_TIME_WITH_NS) - .appendOffsetId() - .toFormatter(); + new DateTimeFormatterBuilder() + .parseCaseInsensitive() + .append(ISO_LOCAL_DATE_WITH_SLASH) + .appendLiteral('T') + .append(ISO_LOCAL_TIME_WITH_NS) + .appendOffsetId() + .toFormatter(); } /** such as '2011.12.03T10:15:30+01:00' or '2011.12.03T10:15:30.123+01:00'. */ @@ -226,13 +226,13 @@ private DatetimeUtils() { static { ISO_OFFSET_DATE_TIME_WITH_DOT = - new DateTimeFormatterBuilder() - .parseCaseInsensitive() - .append(ISO_LOCAL_DATE_WITH_DOT) - .appendLiteral('T') - .append(ISO_LOCAL_TIME_WITH_MS) - .appendOffsetId() - .toFormatter(); + new DateTimeFormatterBuilder() + .parseCaseInsensitive() + .append(ISO_LOCAL_DATE_WITH_DOT) + .appendLiteral('T') + .append(ISO_LOCAL_TIME_WITH_MS) + .appendOffsetId() + .toFormatter(); } /** such as '2011.12.03T10:15:30+01:00' or '2011.12.03T10:15:30.123456+01:00'. */ @@ -240,13 +240,13 @@ private DatetimeUtils() { static { ISO_OFFSET_DATE_TIME_WITH_DOT_US = - new DateTimeFormatterBuilder() - .parseCaseInsensitive() - .append(ISO_LOCAL_DATE_WITH_DOT) - .appendLiteral('T') - .append(ISO_LOCAL_TIME_WITH_US) - .appendOffsetId() - .toFormatter(); + new DateTimeFormatterBuilder() + .parseCaseInsensitive() + .append(ISO_LOCAL_DATE_WITH_DOT) + .appendLiteral('T') + .append(ISO_LOCAL_TIME_WITH_US) + .appendOffsetId() + .toFormatter(); } /** such as '2011.12.03T10:15:30+01:00' or '2011.12.03T10:15:30.123456789+01:00'. */ @@ -254,13 +254,13 @@ private DatetimeUtils() { static { ISO_OFFSET_DATE_TIME_WITH_DOT_NS = - new DateTimeFormatterBuilder() - .parseCaseInsensitive() - .append(ISO_LOCAL_DATE_WITH_DOT) - .appendLiteral('T') - .append(ISO_LOCAL_TIME_WITH_NS) - .appendOffsetId() - .toFormatter(); + new DateTimeFormatterBuilder() + .parseCaseInsensitive() + .append(ISO_LOCAL_DATE_WITH_DOT) + .appendLiteral('T') + .append(ISO_LOCAL_TIME_WITH_NS) + .appendOffsetId() + .toFormatter(); } /** such as '2011-12-03 10:15:30+01:00' or '2011-12-03 10:15:30.123+01:00'. */ @@ -268,13 +268,13 @@ private DatetimeUtils() { static { ISO_OFFSET_DATE_TIME_WITH_SPACE = - new DateTimeFormatterBuilder() - .parseCaseInsensitive() - .append(DateTimeFormatter.ISO_LOCAL_DATE) - .appendLiteral(' ') - .append(ISO_LOCAL_TIME_WITH_MS) - .appendOffsetId() - .toFormatter(); + new DateTimeFormatterBuilder() + .parseCaseInsensitive() + .append(DateTimeFormatter.ISO_LOCAL_DATE) + .appendLiteral(' ') + .append(ISO_LOCAL_TIME_WITH_MS) + .appendOffsetId() + .toFormatter(); } /** such as '2011-12-03 10:15:30+01:00' or '2011-12-03 10:15:30.123456+01:00'. */ @@ -282,13 +282,13 @@ private DatetimeUtils() { static { ISO_OFFSET_DATE_TIME_WITH_SPACE_US = - new DateTimeFormatterBuilder() - .parseCaseInsensitive() - .append(DateTimeFormatter.ISO_LOCAL_DATE) - .appendLiteral(' ') - .append(ISO_LOCAL_TIME_WITH_US) - .appendOffsetId() - .toFormatter(); + new DateTimeFormatterBuilder() + .parseCaseInsensitive() + .append(DateTimeFormatter.ISO_LOCAL_DATE) + .appendLiteral(' ') + .append(ISO_LOCAL_TIME_WITH_US) + .appendOffsetId() + .toFormatter(); } /** such as '2011-12-03 10:15:30+01:00' or '2011-12-03 10:15:30.123456789+01:00'. */ @@ -296,13 +296,13 @@ private DatetimeUtils() { static { ISO_OFFSET_DATE_TIME_WITH_SPACE_NS = - new DateTimeFormatterBuilder() - .parseCaseInsensitive() - .append(DateTimeFormatter.ISO_LOCAL_DATE) - .appendLiteral(' ') - .append(ISO_LOCAL_TIME_WITH_NS) - .appendOffsetId() - .toFormatter(); + new DateTimeFormatterBuilder() + .parseCaseInsensitive() + .append(DateTimeFormatter.ISO_LOCAL_DATE) + .appendLiteral(' ') + .append(ISO_LOCAL_TIME_WITH_NS) + .appendOffsetId() + .toFormatter(); } /** such as '2011/12/03 10:15:30+01:00' or '2011/12/03 10:15:30.123+01:00'. */ @@ -310,13 +310,13 @@ private DatetimeUtils() { static { ISO_OFFSET_DATE_TIME_WITH_SLASH_WITH_SPACE = - new DateTimeFormatterBuilder() - .parseCaseInsensitive() - .append(ISO_LOCAL_DATE_WITH_SLASH) - .appendLiteral(' ') - .append(ISO_LOCAL_TIME_WITH_MS) - .appendOffsetId() - .toFormatter(); + new DateTimeFormatterBuilder() + .parseCaseInsensitive() + .append(ISO_LOCAL_DATE_WITH_SLASH) + .appendLiteral(' ') + .append(ISO_LOCAL_TIME_WITH_MS) + .appendOffsetId() + .toFormatter(); } /** such as '2011/12/03 10:15:30+01:00' or '2011/12/03 10:15:30.123456+01:00'. */ @@ -324,13 +324,13 @@ private DatetimeUtils() { static { ISO_OFFSET_DATE_TIME_WITH_SLASH_WITH_SPACE_US = - new DateTimeFormatterBuilder() - .parseCaseInsensitive() - .append(ISO_LOCAL_DATE_WITH_SLASH) - .appendLiteral(' ') - .append(ISO_LOCAL_TIME_WITH_US) - .appendOffsetId() - .toFormatter(); + new DateTimeFormatterBuilder() + .parseCaseInsensitive() + .append(ISO_LOCAL_DATE_WITH_SLASH) + .appendLiteral(' ') + .append(ISO_LOCAL_TIME_WITH_US) + .appendOffsetId() + .toFormatter(); } /** such as '2011/12/03 10:15:30+01:00' or '2011/12/03 10:15:30.123456789+01:00'. */ @@ -338,13 +338,13 @@ private DatetimeUtils() { static { ISO_OFFSET_DATE_TIME_WITH_SLASH_WITH_SPACE_NS = - new DateTimeFormatterBuilder() - .parseCaseInsensitive() - .append(ISO_LOCAL_DATE_WITH_SLASH) - .appendLiteral(' ') - .append(ISO_LOCAL_TIME_WITH_NS) - .appendOffsetId() - .toFormatter(); + new DateTimeFormatterBuilder() + .parseCaseInsensitive() + .append(ISO_LOCAL_DATE_WITH_SLASH) + .appendLiteral(' ') + .append(ISO_LOCAL_TIME_WITH_NS) + .appendOffsetId() + .toFormatter(); } /** such as '2011.12.03 10:15:30+01:00' or '2011.12.03 10:15:30.123+01:00'. */ @@ -352,13 +352,13 @@ private DatetimeUtils() { static { ISO_OFFSET_DATE_TIME_WITH_DOT_WITH_SPACE = - new DateTimeFormatterBuilder() - .parseCaseInsensitive() - .append(ISO_LOCAL_DATE_WITH_DOT) - .appendLiteral(' ') - .append(ISO_LOCAL_TIME_WITH_MS) - .appendOffsetId() - .toFormatter(); + new DateTimeFormatterBuilder() + .parseCaseInsensitive() + .append(ISO_LOCAL_DATE_WITH_DOT) + .appendLiteral(' ') + .append(ISO_LOCAL_TIME_WITH_MS) + .appendOffsetId() + .toFormatter(); } /** such as '2011.12.03 10:15:30+01:00' or '2011.12.03 10:15:30.123456+01:00'. */ @@ -366,13 +366,13 @@ private DatetimeUtils() { static { ISO_OFFSET_DATE_TIME_WITH_DOT_WITH_SPACE_US = - new DateTimeFormatterBuilder() - .parseCaseInsensitive() - .append(ISO_LOCAL_DATE_WITH_DOT) - .appendLiteral(' ') - .append(ISO_LOCAL_TIME_WITH_US) - .appendOffsetId() - .toFormatter(); + new DateTimeFormatterBuilder() + .parseCaseInsensitive() + .append(ISO_LOCAL_DATE_WITH_DOT) + .appendLiteral(' ') + .append(ISO_LOCAL_TIME_WITH_US) + .appendOffsetId() + .toFormatter(); } /** such as '2011.12.03 10:15:30+01:00' or '2011.12.03 10:15:30.123456789+01:00'. */ @@ -380,82 +380,82 @@ private DatetimeUtils() { static { ISO_OFFSET_DATE_TIME_WITH_DOT_WITH_SPACE_NS = - new DateTimeFormatterBuilder() - .parseCaseInsensitive() - .append(ISO_LOCAL_DATE_WITH_DOT) - .appendLiteral(' ') - .append(ISO_LOCAL_TIME_WITH_NS) - .appendOffsetId() - .toFormatter(); + new DateTimeFormatterBuilder() + .parseCaseInsensitive() + .append(ISO_LOCAL_DATE_WITH_DOT) + .appendLiteral(' ') + .append(ISO_LOCAL_TIME_WITH_NS) + .appendOffsetId() + .toFormatter(); } public static final DateTimeFormatter formatter = - new DateTimeFormatterBuilder() - /** - * The ISO date-time formatter that formats or parses a date-time with an offset, such as - * '2011-12-03T10:15:30+01:00' or '2011-12-03T10:15:30.123+01:00'. - */ - .appendOptional(ISO_OFFSET_DATE_TIME_WITH_MS) + new DateTimeFormatterBuilder() + /** + * The ISO date-time formatter that formats or parses a date-time with an offset, such as + * '2011-12-03T10:15:30+01:00' or '2011-12-03T10:15:30.123+01:00'. + */ + .appendOptional(ISO_OFFSET_DATE_TIME_WITH_MS) - /** such as '2011-12-03T10:15:30+01:00' or '2011-12-03T10:15:30.123456+01:00'. */ - .appendOptional(ISO_OFFSET_DATE_TIME_WITH_US) + /** such as '2011-12-03T10:15:30+01:00' or '2011-12-03T10:15:30.123456+01:00'. */ + .appendOptional(ISO_OFFSET_DATE_TIME_WITH_US) - /** such as '2011-12-03T10:15:30+01:00' or '2011-12-03T10:15:30.123456789+01:00'. */ - .appendOptional(ISO_OFFSET_DATE_TIME_WITH_NS) + /** such as '2011-12-03T10:15:30+01:00' or '2011-12-03T10:15:30.123456789+01:00'. */ + .appendOptional(ISO_OFFSET_DATE_TIME_WITH_NS) - /** such as '2011/12/03T10:15:30+01:00' or '2011/12/03T10:15:30.123+01:00'. */ - .appendOptional(ISO_OFFSET_DATE_TIME_WITH_SLASH) + /** such as '2011/12/03T10:15:30+01:00' or '2011/12/03T10:15:30.123+01:00'. */ + .appendOptional(ISO_OFFSET_DATE_TIME_WITH_SLASH) - /** such as '2011/12/03T10:15:30+01:00' or '2011/12/03T10:15:30.123456+01:00'. */ - .appendOptional(ISO_OFFSET_DATE_TIME_WITH_SLASH_US) + /** such as '2011/12/03T10:15:30+01:00' or '2011/12/03T10:15:30.123456+01:00'. */ + .appendOptional(ISO_OFFSET_DATE_TIME_WITH_SLASH_US) - /** such as '2011/12/03T10:15:30+01:00' or '2011/12/03T10:15:30.123456789+01:00'. */ - .appendOptional(ISO_OFFSET_DATE_TIME_WITH_SLASH_NS) + /** such as '2011/12/03T10:15:30+01:00' or '2011/12/03T10:15:30.123456789+01:00'. */ + .appendOptional(ISO_OFFSET_DATE_TIME_WITH_SLASH_NS) - /** such as '2011.12.03T10:15:30+01:00' or '2011.12.03T10:15:30.123+01:00'. */ - .appendOptional(ISO_OFFSET_DATE_TIME_WITH_DOT) + /** such as '2011.12.03T10:15:30+01:00' or '2011.12.03T10:15:30.123+01:00'. */ + .appendOptional(ISO_OFFSET_DATE_TIME_WITH_DOT) - /** such as '2011.12.03T10:15:30+01:00' or '2011.12.03T10:15:30.123456+01:00'. */ - .appendOptional(ISO_OFFSET_DATE_TIME_WITH_DOT_US) + /** such as '2011.12.03T10:15:30+01:00' or '2011.12.03T10:15:30.123456+01:00'. */ + .appendOptional(ISO_OFFSET_DATE_TIME_WITH_DOT_US) - /** such as '2011.12.03T10:15:30+01:00' or '2011.12.03T10:15:30.123456789+01:00'. */ - .appendOptional(ISO_OFFSET_DATE_TIME_WITH_DOT_NS) + /** such as '2011.12.03T10:15:30+01:00' or '2011.12.03T10:15:30.123456789+01:00'. */ + .appendOptional(ISO_OFFSET_DATE_TIME_WITH_DOT_NS) - /** such as '2011-12-03 10:15:30+01:00' or '2011-12-03 10:15:30.123+01:00'. */ - .appendOptional(ISO_OFFSET_DATE_TIME_WITH_SPACE) + /** such as '2011-12-03 10:15:30+01:00' or '2011-12-03 10:15:30.123+01:00'. */ + .appendOptional(ISO_OFFSET_DATE_TIME_WITH_SPACE) - /** such as '2011-12-03 10:15:30+01:00' or '2011-12-03 10:15:30.123456+01:00'. */ - .appendOptional(ISO_OFFSET_DATE_TIME_WITH_SPACE_US) + /** such as '2011-12-03 10:15:30+01:00' or '2011-12-03 10:15:30.123456+01:00'. */ + .appendOptional(ISO_OFFSET_DATE_TIME_WITH_SPACE_US) - /** such as '2011-12-03 10:15:30+01:00' or '2011-12-03 10:15:30.123456789+01:00'. */ - .appendOptional(ISO_OFFSET_DATE_TIME_WITH_SPACE_NS) + /** such as '2011-12-03 10:15:30+01:00' or '2011-12-03 10:15:30.123456789+01:00'. */ + .appendOptional(ISO_OFFSET_DATE_TIME_WITH_SPACE_NS) - /** such as '2011/12/03 10:15:30+01:00' or '2011/12/03 10:15:30.123+01:00'. */ - .appendOptional(ISO_OFFSET_DATE_TIME_WITH_SLASH_WITH_SPACE) + /** such as '2011/12/03 10:15:30+01:00' or '2011/12/03 10:15:30.123+01:00'. */ + .appendOptional(ISO_OFFSET_DATE_TIME_WITH_SLASH_WITH_SPACE) - /** such as '2011/12/03 10:15:30+01:00' or '2011/12/03 10:15:30.123456+01:00'. */ - .appendOptional(ISO_OFFSET_DATE_TIME_WITH_SLASH_WITH_SPACE_US) + /** such as '2011/12/03 10:15:30+01:00' or '2011/12/03 10:15:30.123456+01:00'. */ + .appendOptional(ISO_OFFSET_DATE_TIME_WITH_SLASH_WITH_SPACE_US) - /** such as '2011/12/03 10:15:30+01:00' or '2011/12/03 10:15:30.123456789+01:00'. */ - .appendOptional(ISO_OFFSET_DATE_TIME_WITH_SLASH_WITH_SPACE_NS) + /** such as '2011/12/03 10:15:30+01:00' or '2011/12/03 10:15:30.123456789+01:00'. */ + .appendOptional(ISO_OFFSET_DATE_TIME_WITH_SLASH_WITH_SPACE_NS) - /** such as '2011.12.03 10:15:30+01:00' or '2011.12.03 10:15:30.123+01:00'. */ - .appendOptional(ISO_OFFSET_DATE_TIME_WITH_DOT_WITH_SPACE) + /** such as '2011.12.03 10:15:30+01:00' or '2011.12.03 10:15:30.123+01:00'. */ + .appendOptional(ISO_OFFSET_DATE_TIME_WITH_DOT_WITH_SPACE) - /** such as '2011.12.03 10:15:30+01:00' or '2011.12.03 10:15:30.123456+01:00'. */ - .appendOptional(ISO_OFFSET_DATE_TIME_WITH_DOT_WITH_SPACE_US) + /** such as '2011.12.03 10:15:30+01:00' or '2011.12.03 10:15:30.123456+01:00'. */ + .appendOptional(ISO_OFFSET_DATE_TIME_WITH_DOT_WITH_SPACE_US) - /** such as '2011.12.03 10:15:30+01:00' or '2011.12.03 10:15:30.123456789+01:00'. */ - .appendOptional(ISO_OFFSET_DATE_TIME_WITH_DOT_WITH_SPACE_NS) - .toFormatter(); + /** such as '2011.12.03 10:15:30+01:00' or '2011.12.03 10:15:30.123456789+01:00'. */ + .appendOptional(ISO_OFFSET_DATE_TIME_WITH_DOT_WITH_SPACE_NS) + .toFormatter(); public static long convertDatetimeStrToLong(String str, ZoneId zoneId) - throws LogicalOperatorException { + throws LogicalOperatorException { return convertDatetimeStrToLong(str, toZoneOffset(zoneId), 0); } public static long getInstantWithPrecision(String str, String timestampPrecision) - throws LogicalOperatorException { + throws LogicalOperatorException { try { ZonedDateTime zonedDateTime = ZonedDateTime.parse(str, formatter); Instant instant = zonedDateTime.toInstant(); @@ -481,30 +481,30 @@ public static long getInstantWithPrecision(String str, String timestampPrecision /** convert date time string to millisecond, microsecond or nanosecond. */ public static long convertDatetimeStrToLong(String str, ZoneOffset offset, int depth) - throws LogicalOperatorException { + throws LogicalOperatorException { String timestampPrecision = IoTDBDescriptor.getInstance().getConfig().getTimestampPrecision(); if (depth >= 2) { throw new DateTimeException( - String.format( - "Failed to convert %s to millisecond, zone offset is %s, " - + "please input like 2011-12-03T10:15:30 or 2011-12-03T10:15:30+01:00", - str, offset)); + String.format( + "Failed to convert %s to millisecond, zone offset is %s, " + + "please input like 2011-12-03T10:15:30 or 2011-12-03T10:15:30+01:00", + str, offset)); } if (str.contains("Z")) { return convertDatetimeStrToLong(str.substring(0, str.indexOf('Z')) + "+00:00", offset, depth); } else if (str.length() == 10) { return convertDatetimeStrToLong(str + "T00:00:00", offset, depth); } else if (str.length() - str.lastIndexOf('+') != 6 - && str.length() - str.lastIndexOf('-') != 6) { + && str.length() - str.lastIndexOf('-') != 6) { return convertDatetimeStrToLong(str + offset, offset, depth + 1); } else if (str.contains("[") || str.contains("]")) { throw new DateTimeException( - String.format( - "%s with [time-region] at end is not supported now, " - + "please input like 2011-12-03T10:15:30 or 2011-12-03T10:15:30+01:00", - str)); + String.format( + "%s with [time-region] at end is not supported now, " + + "please input like 2011-12-03T10:15:30 or 2011-12-03T10:15:30+01:00", + str)); } return getInstantWithPrecision(str, timestampPrecision); } @@ -527,7 +527,7 @@ public static long convertDurationStrToLong(String duration, String timestampPre public static long convertDurationStrToLong(long currentTime, String duration) { return convertDurationStrToLong( - currentTime, duration, IoTDBDescriptor.getInstance().getConfig().getTimestampPrecision()); + currentTime, duration, IoTDBDescriptor.getInstance().getConfig().getTimestampPrecision()); } /** @@ -537,7 +537,7 @@ public static long convertDurationStrToLong(long currentTime, String duration) { * @return time in milliseconds, microseconds, or nanoseconds depending on the profile */ public static long convertDurationStrToLong( - long currentTime, String duration, String timestampPrecision) { + long currentTime, String duration, String timestampPrecision) { long total = 0; long temp = 0; for (int i = 0; i < duration.length(); i++) { @@ -553,11 +553,11 @@ public static long convertDurationStrToLong( unit += duration.charAt(i); } total += - DatetimeUtils.convertDurationStrToLong( - currentTime == -1 ? -1 : currentTime + total, - temp, - unit.toLowerCase(), - timestampPrecision); + DatetimeUtils.convertDurationStrToLong( + currentTime == -1 ? -1 : currentTime + total, + temp, + unit.toLowerCase(), + timestampPrecision); temp = 0; } } @@ -566,13 +566,13 @@ public static long convertDurationStrToLong( @TestOnly public static long convertDurationStrToLongForTest( - long value, String unit, String timestampPrecision) { + long value, String unit, String timestampPrecision) { return convertDurationStrToLong(-1, value, unit, timestampPrecision); } /** convert duration string to millisecond, microsecond or nanosecond. */ public static long convertDurationStrToLong( - long currentTime, long value, String unit, String timestampPrecision) { + long currentTime, long value, String unit, String timestampPrecision) { DurationUnit durationUnit = DurationUnit.valueOf(unit); long res = value; switch (durationUnit) { @@ -584,9 +584,7 @@ public static long convertDurationStrToLong( res *= 30 * 86_400_000L; } else { Calendar calendar = Calendar.getInstance(); - TimeZone tz = TimeZone.getTimeZone("GMT+8"); - // calendar.setTimeZone(SessionManager.getInstance().getCurrSessionTimeZone()); - calendar.setTimeZone(tz); + calendar.setTimeZone(SessionManager.getInstance().getCurrSessionTimeZone()); calendar.setTimeInMillis(currentTime); calendar.add(Calendar.MONTH, (int) (value)); res = calendar.getTimeInMillis() - currentTime; From 209f0548dc606a033277bebe2037e904af325637 Mon Sep 17 00:00:00 2001 From: Yuting Date: Wed, 14 Jul 2021 00:01:50 -0500 Subject: [PATCH 10/28] merge master --- .../iotdb/db/qp/utils/DatetimeUtils.java | 484 +++++++++--------- 1 file changed, 242 insertions(+), 242 deletions(-) diff --git a/server/src/main/java/org/apache/iotdb/db/qp/utils/DatetimeUtils.java b/server/src/main/java/org/apache/iotdb/db/qp/utils/DatetimeUtils.java index 1ee5477badee0..d360e1e54635b 100644 --- a/server/src/main/java/org/apache/iotdb/db/qp/utils/DatetimeUtils.java +++ b/server/src/main/java/org/apache/iotdb/db/qp/utils/DatetimeUtils.java @@ -46,13 +46,13 @@ private DatetimeUtils() { static { ISO_LOCAL_DATE_WIDTH_1_2 = - new DateTimeFormatterBuilder() - .appendValue(ChronoField.YEAR, 4, 10, SignStyle.EXCEEDS_PAD) - .appendLiteral('-') - .appendValue(ChronoField.MONTH_OF_YEAR, 1, 2, SignStyle.NEVER) - .appendLiteral('-') - .appendValue(ChronoField.DAY_OF_MONTH, 1, 2, SignStyle.NEVER) - .toFormatter(); + new DateTimeFormatterBuilder() + .appendValue(ChronoField.YEAR, 4, 10, SignStyle.EXCEEDS_PAD) + .appendLiteral('-') + .appendValue(ChronoField.MONTH_OF_YEAR, 1, 2, SignStyle.NEVER) + .appendLiteral('-') + .appendValue(ChronoField.DAY_OF_MONTH, 1, 2, SignStyle.NEVER) + .toFormatter(); } /** such as '2011/12/03'. */ @@ -60,13 +60,13 @@ private DatetimeUtils() { static { ISO_LOCAL_DATE_WITH_SLASH = - new DateTimeFormatterBuilder() - .appendValue(ChronoField.YEAR, 4, 10, SignStyle.EXCEEDS_PAD) - .appendLiteral('/') - .appendValue(ChronoField.MONTH_OF_YEAR, 1, 2, SignStyle.NEVER) - .appendLiteral('/') - .appendValue(ChronoField.DAY_OF_MONTH, 1, 2, SignStyle.NEVER) - .toFormatter(); + new DateTimeFormatterBuilder() + .appendValue(ChronoField.YEAR, 4, 10, SignStyle.EXCEEDS_PAD) + .appendLiteral('/') + .appendValue(ChronoField.MONTH_OF_YEAR, 1, 2, SignStyle.NEVER) + .appendLiteral('/') + .appendValue(ChronoField.DAY_OF_MONTH, 1, 2, SignStyle.NEVER) + .toFormatter(); } /** such as '2011.12.03'. */ @@ -74,13 +74,13 @@ private DatetimeUtils() { static { ISO_LOCAL_DATE_WITH_DOT = - new DateTimeFormatterBuilder() - .appendValue(ChronoField.YEAR, 4, 10, SignStyle.EXCEEDS_PAD) - .appendLiteral('.') - .appendValue(ChronoField.MONTH_OF_YEAR, 1, 2, SignStyle.NEVER) - .appendLiteral('.') - .appendValue(ChronoField.DAY_OF_MONTH, 1, 2, SignStyle.NEVER) - .toFormatter(); + new DateTimeFormatterBuilder() + .appendValue(ChronoField.YEAR, 4, 10, SignStyle.EXCEEDS_PAD) + .appendLiteral('.') + .appendValue(ChronoField.MONTH_OF_YEAR, 1, 2, SignStyle.NEVER) + .appendLiteral('.') + .appendValue(ChronoField.DAY_OF_MONTH, 1, 2, SignStyle.NEVER) + .toFormatter(); } /** such as '10:15:30' or '10:15:30.123'. */ @@ -88,17 +88,17 @@ private DatetimeUtils() { static { ISO_LOCAL_TIME_WITH_MS = - new DateTimeFormatterBuilder() - .appendValue(ChronoField.HOUR_OF_DAY, 2) - .appendLiteral(':') - .appendValue(ChronoField.MINUTE_OF_HOUR, 2) - .appendLiteral(':') - .appendValue(ChronoField.SECOND_OF_MINUTE, 2) - .optionalStart() - .appendLiteral('.') - .appendValue(ChronoField.MILLI_OF_SECOND, 3) - .optionalEnd() - .toFormatter(); + new DateTimeFormatterBuilder() + .appendValue(ChronoField.HOUR_OF_DAY, 2) + .appendLiteral(':') + .appendValue(ChronoField.MINUTE_OF_HOUR, 2) + .appendLiteral(':') + .appendValue(ChronoField.SECOND_OF_MINUTE, 2) + .optionalStart() + .appendLiteral('.') + .appendValue(ChronoField.MILLI_OF_SECOND, 3) + .optionalEnd() + .toFormatter(); } /** such as '10:15:30' or '10:15:30.123456'. */ @@ -106,17 +106,17 @@ private DatetimeUtils() { static { ISO_LOCAL_TIME_WITH_US = - new DateTimeFormatterBuilder() - .appendValue(ChronoField.HOUR_OF_DAY, 2) - .appendLiteral(':') - .appendValue(ChronoField.MINUTE_OF_HOUR, 2) - .appendLiteral(':') - .appendValue(ChronoField.SECOND_OF_MINUTE, 2) - .optionalStart() - .appendLiteral('.') - .appendValue(ChronoField.MICRO_OF_SECOND, 6) - .optionalEnd() - .toFormatter(); + new DateTimeFormatterBuilder() + .appendValue(ChronoField.HOUR_OF_DAY, 2) + .appendLiteral(':') + .appendValue(ChronoField.MINUTE_OF_HOUR, 2) + .appendLiteral(':') + .appendValue(ChronoField.SECOND_OF_MINUTE, 2) + .optionalStart() + .appendLiteral('.') + .appendValue(ChronoField.MICRO_OF_SECOND, 6) + .optionalEnd() + .toFormatter(); } /** such as '10:15:30' or '10:15:30.123456789'. */ @@ -124,17 +124,17 @@ private DatetimeUtils() { static { ISO_LOCAL_TIME_WITH_NS = - new DateTimeFormatterBuilder() - .appendValue(ChronoField.HOUR_OF_DAY, 2) - .appendLiteral(':') - .appendValue(ChronoField.MINUTE_OF_HOUR, 2) - .appendLiteral(':') - .appendValue(ChronoField.SECOND_OF_MINUTE, 2) - .optionalStart() - .appendLiteral('.') - .appendValue(ChronoField.NANO_OF_SECOND, 9) - .optionalEnd() - .toFormatter(); + new DateTimeFormatterBuilder() + .appendValue(ChronoField.HOUR_OF_DAY, 2) + .appendLiteral(':') + .appendValue(ChronoField.MINUTE_OF_HOUR, 2) + .appendLiteral(':') + .appendValue(ChronoField.SECOND_OF_MINUTE, 2) + .optionalStart() + .appendLiteral('.') + .appendValue(ChronoField.NANO_OF_SECOND, 9) + .optionalEnd() + .toFormatter(); } /** such as '2011-12-03T10:15:30+01:00' or '2011-12-03T10:15:30.123+01:00'. */ @@ -142,13 +142,13 @@ private DatetimeUtils() { static { ISO_OFFSET_DATE_TIME_WITH_MS = - new DateTimeFormatterBuilder() - .parseCaseInsensitive() - .append(ISO_LOCAL_DATE_WIDTH_1_2) - .appendLiteral('T') - .append(ISO_LOCAL_TIME_WITH_MS) - .appendOffsetId() - .toFormatter(); + new DateTimeFormatterBuilder() + .parseCaseInsensitive() + .append(ISO_LOCAL_DATE_WIDTH_1_2) + .appendLiteral('T') + .append(ISO_LOCAL_TIME_WITH_MS) + .appendOffsetId() + .toFormatter(); } /** such as '2011-12-03T10:15:30+01:00' or '2011-12-03T10:15:30.123456+01:00'. */ @@ -156,13 +156,13 @@ private DatetimeUtils() { static { ISO_OFFSET_DATE_TIME_WITH_US = - new DateTimeFormatterBuilder() - .parseCaseInsensitive() - .append(ISO_LOCAL_DATE_WIDTH_1_2) - .appendLiteral('T') - .append(ISO_LOCAL_TIME_WITH_US) - .appendOffsetId() - .toFormatter(); + new DateTimeFormatterBuilder() + .parseCaseInsensitive() + .append(ISO_LOCAL_DATE_WIDTH_1_2) + .appendLiteral('T') + .append(ISO_LOCAL_TIME_WITH_US) + .appendOffsetId() + .toFormatter(); } /** such as '2011-12-03T10:15:30+01:00' or '2011-12-03T10:15:30.123456789+01:00'. */ @@ -170,13 +170,13 @@ private DatetimeUtils() { static { ISO_OFFSET_DATE_TIME_WITH_NS = - new DateTimeFormatterBuilder() - .parseCaseInsensitive() - .append(ISO_LOCAL_DATE_WIDTH_1_2) - .appendLiteral('T') - .append(ISO_LOCAL_TIME_WITH_NS) - .appendOffsetId() - .toFormatter(); + new DateTimeFormatterBuilder() + .parseCaseInsensitive() + .append(ISO_LOCAL_DATE_WIDTH_1_2) + .appendLiteral('T') + .append(ISO_LOCAL_TIME_WITH_NS) + .appendOffsetId() + .toFormatter(); } /** such as '2011/12/03T10:15:30+01:00' or '2011/12/03T10:15:30.123+01:00'. */ @@ -184,13 +184,13 @@ private DatetimeUtils() { static { ISO_OFFSET_DATE_TIME_WITH_SLASH = - new DateTimeFormatterBuilder() - .parseCaseInsensitive() - .append(ISO_LOCAL_DATE_WITH_SLASH) - .appendLiteral('T') - .append(ISO_LOCAL_TIME_WITH_MS) - .appendOffsetId() - .toFormatter(); + new DateTimeFormatterBuilder() + .parseCaseInsensitive() + .append(ISO_LOCAL_DATE_WITH_SLASH) + .appendLiteral('T') + .append(ISO_LOCAL_TIME_WITH_MS) + .appendOffsetId() + .toFormatter(); } /** such as '2011/12/03T10:15:30+01:00' or '2011/12/03T10:15:30.123456+01:00'. */ @@ -198,13 +198,13 @@ private DatetimeUtils() { static { ISO_OFFSET_DATE_TIME_WITH_SLASH_US = - new DateTimeFormatterBuilder() - .parseCaseInsensitive() - .append(ISO_LOCAL_DATE_WITH_SLASH) - .appendLiteral('T') - .append(ISO_LOCAL_TIME_WITH_US) - .appendOffsetId() - .toFormatter(); + new DateTimeFormatterBuilder() + .parseCaseInsensitive() + .append(ISO_LOCAL_DATE_WITH_SLASH) + .appendLiteral('T') + .append(ISO_LOCAL_TIME_WITH_US) + .appendOffsetId() + .toFormatter(); } /** such as '2011/12/03T10:15:30+01:00' or '2011/12/03T10:15:30.123456789+01:00'. */ @@ -212,13 +212,13 @@ private DatetimeUtils() { static { ISO_OFFSET_DATE_TIME_WITH_SLASH_NS = - new DateTimeFormatterBuilder() - .parseCaseInsensitive() - .append(ISO_LOCAL_DATE_WITH_SLASH) - .appendLiteral('T') - .append(ISO_LOCAL_TIME_WITH_NS) - .appendOffsetId() - .toFormatter(); + new DateTimeFormatterBuilder() + .parseCaseInsensitive() + .append(ISO_LOCAL_DATE_WITH_SLASH) + .appendLiteral('T') + .append(ISO_LOCAL_TIME_WITH_NS) + .appendOffsetId() + .toFormatter(); } /** such as '2011.12.03T10:15:30+01:00' or '2011.12.03T10:15:30.123+01:00'. */ @@ -226,13 +226,13 @@ private DatetimeUtils() { static { ISO_OFFSET_DATE_TIME_WITH_DOT = - new DateTimeFormatterBuilder() - .parseCaseInsensitive() - .append(ISO_LOCAL_DATE_WITH_DOT) - .appendLiteral('T') - .append(ISO_LOCAL_TIME_WITH_MS) - .appendOffsetId() - .toFormatter(); + new DateTimeFormatterBuilder() + .parseCaseInsensitive() + .append(ISO_LOCAL_DATE_WITH_DOT) + .appendLiteral('T') + .append(ISO_LOCAL_TIME_WITH_MS) + .appendOffsetId() + .toFormatter(); } /** such as '2011.12.03T10:15:30+01:00' or '2011.12.03T10:15:30.123456+01:00'. */ @@ -240,13 +240,13 @@ private DatetimeUtils() { static { ISO_OFFSET_DATE_TIME_WITH_DOT_US = - new DateTimeFormatterBuilder() - .parseCaseInsensitive() - .append(ISO_LOCAL_DATE_WITH_DOT) - .appendLiteral('T') - .append(ISO_LOCAL_TIME_WITH_US) - .appendOffsetId() - .toFormatter(); + new DateTimeFormatterBuilder() + .parseCaseInsensitive() + .append(ISO_LOCAL_DATE_WITH_DOT) + .appendLiteral('T') + .append(ISO_LOCAL_TIME_WITH_US) + .appendOffsetId() + .toFormatter(); } /** such as '2011.12.03T10:15:30+01:00' or '2011.12.03T10:15:30.123456789+01:00'. */ @@ -254,13 +254,13 @@ private DatetimeUtils() { static { ISO_OFFSET_DATE_TIME_WITH_DOT_NS = - new DateTimeFormatterBuilder() - .parseCaseInsensitive() - .append(ISO_LOCAL_DATE_WITH_DOT) - .appendLiteral('T') - .append(ISO_LOCAL_TIME_WITH_NS) - .appendOffsetId() - .toFormatter(); + new DateTimeFormatterBuilder() + .parseCaseInsensitive() + .append(ISO_LOCAL_DATE_WITH_DOT) + .appendLiteral('T') + .append(ISO_LOCAL_TIME_WITH_NS) + .appendOffsetId() + .toFormatter(); } /** such as '2011-12-03 10:15:30+01:00' or '2011-12-03 10:15:30.123+01:00'. */ @@ -268,13 +268,13 @@ private DatetimeUtils() { static { ISO_OFFSET_DATE_TIME_WITH_SPACE = - new DateTimeFormatterBuilder() - .parseCaseInsensitive() - .append(DateTimeFormatter.ISO_LOCAL_DATE) - .appendLiteral(' ') - .append(ISO_LOCAL_TIME_WITH_MS) - .appendOffsetId() - .toFormatter(); + new DateTimeFormatterBuilder() + .parseCaseInsensitive() + .append(DateTimeFormatter.ISO_LOCAL_DATE) + .appendLiteral(' ') + .append(ISO_LOCAL_TIME_WITH_MS) + .appendOffsetId() + .toFormatter(); } /** such as '2011-12-03 10:15:30+01:00' or '2011-12-03 10:15:30.123456+01:00'. */ @@ -282,13 +282,13 @@ private DatetimeUtils() { static { ISO_OFFSET_DATE_TIME_WITH_SPACE_US = - new DateTimeFormatterBuilder() - .parseCaseInsensitive() - .append(DateTimeFormatter.ISO_LOCAL_DATE) - .appendLiteral(' ') - .append(ISO_LOCAL_TIME_WITH_US) - .appendOffsetId() - .toFormatter(); + new DateTimeFormatterBuilder() + .parseCaseInsensitive() + .append(DateTimeFormatter.ISO_LOCAL_DATE) + .appendLiteral(' ') + .append(ISO_LOCAL_TIME_WITH_US) + .appendOffsetId() + .toFormatter(); } /** such as '2011-12-03 10:15:30+01:00' or '2011-12-03 10:15:30.123456789+01:00'. */ @@ -296,13 +296,13 @@ private DatetimeUtils() { static { ISO_OFFSET_DATE_TIME_WITH_SPACE_NS = - new DateTimeFormatterBuilder() - .parseCaseInsensitive() - .append(DateTimeFormatter.ISO_LOCAL_DATE) - .appendLiteral(' ') - .append(ISO_LOCAL_TIME_WITH_NS) - .appendOffsetId() - .toFormatter(); + new DateTimeFormatterBuilder() + .parseCaseInsensitive() + .append(DateTimeFormatter.ISO_LOCAL_DATE) + .appendLiteral(' ') + .append(ISO_LOCAL_TIME_WITH_NS) + .appendOffsetId() + .toFormatter(); } /** such as '2011/12/03 10:15:30+01:00' or '2011/12/03 10:15:30.123+01:00'. */ @@ -310,13 +310,13 @@ private DatetimeUtils() { static { ISO_OFFSET_DATE_TIME_WITH_SLASH_WITH_SPACE = - new DateTimeFormatterBuilder() - .parseCaseInsensitive() - .append(ISO_LOCAL_DATE_WITH_SLASH) - .appendLiteral(' ') - .append(ISO_LOCAL_TIME_WITH_MS) - .appendOffsetId() - .toFormatter(); + new DateTimeFormatterBuilder() + .parseCaseInsensitive() + .append(ISO_LOCAL_DATE_WITH_SLASH) + .appendLiteral(' ') + .append(ISO_LOCAL_TIME_WITH_MS) + .appendOffsetId() + .toFormatter(); } /** such as '2011/12/03 10:15:30+01:00' or '2011/12/03 10:15:30.123456+01:00'. */ @@ -324,13 +324,13 @@ private DatetimeUtils() { static { ISO_OFFSET_DATE_TIME_WITH_SLASH_WITH_SPACE_US = - new DateTimeFormatterBuilder() - .parseCaseInsensitive() - .append(ISO_LOCAL_DATE_WITH_SLASH) - .appendLiteral(' ') - .append(ISO_LOCAL_TIME_WITH_US) - .appendOffsetId() - .toFormatter(); + new DateTimeFormatterBuilder() + .parseCaseInsensitive() + .append(ISO_LOCAL_DATE_WITH_SLASH) + .appendLiteral(' ') + .append(ISO_LOCAL_TIME_WITH_US) + .appendOffsetId() + .toFormatter(); } /** such as '2011/12/03 10:15:30+01:00' or '2011/12/03 10:15:30.123456789+01:00'. */ @@ -338,13 +338,13 @@ private DatetimeUtils() { static { ISO_OFFSET_DATE_TIME_WITH_SLASH_WITH_SPACE_NS = - new DateTimeFormatterBuilder() - .parseCaseInsensitive() - .append(ISO_LOCAL_DATE_WITH_SLASH) - .appendLiteral(' ') - .append(ISO_LOCAL_TIME_WITH_NS) - .appendOffsetId() - .toFormatter(); + new DateTimeFormatterBuilder() + .parseCaseInsensitive() + .append(ISO_LOCAL_DATE_WITH_SLASH) + .appendLiteral(' ') + .append(ISO_LOCAL_TIME_WITH_NS) + .appendOffsetId() + .toFormatter(); } /** such as '2011.12.03 10:15:30+01:00' or '2011.12.03 10:15:30.123+01:00'. */ @@ -352,13 +352,13 @@ private DatetimeUtils() { static { ISO_OFFSET_DATE_TIME_WITH_DOT_WITH_SPACE = - new DateTimeFormatterBuilder() - .parseCaseInsensitive() - .append(ISO_LOCAL_DATE_WITH_DOT) - .appendLiteral(' ') - .append(ISO_LOCAL_TIME_WITH_MS) - .appendOffsetId() - .toFormatter(); + new DateTimeFormatterBuilder() + .parseCaseInsensitive() + .append(ISO_LOCAL_DATE_WITH_DOT) + .appendLiteral(' ') + .append(ISO_LOCAL_TIME_WITH_MS) + .appendOffsetId() + .toFormatter(); } /** such as '2011.12.03 10:15:30+01:00' or '2011.12.03 10:15:30.123456+01:00'. */ @@ -366,13 +366,13 @@ private DatetimeUtils() { static { ISO_OFFSET_DATE_TIME_WITH_DOT_WITH_SPACE_US = - new DateTimeFormatterBuilder() - .parseCaseInsensitive() - .append(ISO_LOCAL_DATE_WITH_DOT) - .appendLiteral(' ') - .append(ISO_LOCAL_TIME_WITH_US) - .appendOffsetId() - .toFormatter(); + new DateTimeFormatterBuilder() + .parseCaseInsensitive() + .append(ISO_LOCAL_DATE_WITH_DOT) + .appendLiteral(' ') + .append(ISO_LOCAL_TIME_WITH_US) + .appendOffsetId() + .toFormatter(); } /** such as '2011.12.03 10:15:30+01:00' or '2011.12.03 10:15:30.123456789+01:00'. */ @@ -380,82 +380,82 @@ private DatetimeUtils() { static { ISO_OFFSET_DATE_TIME_WITH_DOT_WITH_SPACE_NS = - new DateTimeFormatterBuilder() - .parseCaseInsensitive() - .append(ISO_LOCAL_DATE_WITH_DOT) - .appendLiteral(' ') - .append(ISO_LOCAL_TIME_WITH_NS) - .appendOffsetId() - .toFormatter(); + new DateTimeFormatterBuilder() + .parseCaseInsensitive() + .append(ISO_LOCAL_DATE_WITH_DOT) + .appendLiteral(' ') + .append(ISO_LOCAL_TIME_WITH_NS) + .appendOffsetId() + .toFormatter(); } public static final DateTimeFormatter formatter = - new DateTimeFormatterBuilder() - /** - * The ISO date-time formatter that formats or parses a date-time with an offset, such as - * '2011-12-03T10:15:30+01:00' or '2011-12-03T10:15:30.123+01:00'. - */ - .appendOptional(ISO_OFFSET_DATE_TIME_WITH_MS) + new DateTimeFormatterBuilder() + /** + * The ISO date-time formatter that formats or parses a date-time with an offset, such as + * '2011-12-03T10:15:30+01:00' or '2011-12-03T10:15:30.123+01:00'. + */ + .appendOptional(ISO_OFFSET_DATE_TIME_WITH_MS) - /** such as '2011-12-03T10:15:30+01:00' or '2011-12-03T10:15:30.123456+01:00'. */ - .appendOptional(ISO_OFFSET_DATE_TIME_WITH_US) + /** such as '2011-12-03T10:15:30+01:00' or '2011-12-03T10:15:30.123456+01:00'. */ + .appendOptional(ISO_OFFSET_DATE_TIME_WITH_US) - /** such as '2011-12-03T10:15:30+01:00' or '2011-12-03T10:15:30.123456789+01:00'. */ - .appendOptional(ISO_OFFSET_DATE_TIME_WITH_NS) + /** such as '2011-12-03T10:15:30+01:00' or '2011-12-03T10:15:30.123456789+01:00'. */ + .appendOptional(ISO_OFFSET_DATE_TIME_WITH_NS) - /** such as '2011/12/03T10:15:30+01:00' or '2011/12/03T10:15:30.123+01:00'. */ - .appendOptional(ISO_OFFSET_DATE_TIME_WITH_SLASH) + /** such as '2011/12/03T10:15:30+01:00' or '2011/12/03T10:15:30.123+01:00'. */ + .appendOptional(ISO_OFFSET_DATE_TIME_WITH_SLASH) - /** such as '2011/12/03T10:15:30+01:00' or '2011/12/03T10:15:30.123456+01:00'. */ - .appendOptional(ISO_OFFSET_DATE_TIME_WITH_SLASH_US) + /** such as '2011/12/03T10:15:30+01:00' or '2011/12/03T10:15:30.123456+01:00'. */ + .appendOptional(ISO_OFFSET_DATE_TIME_WITH_SLASH_US) - /** such as '2011/12/03T10:15:30+01:00' or '2011/12/03T10:15:30.123456789+01:00'. */ - .appendOptional(ISO_OFFSET_DATE_TIME_WITH_SLASH_NS) + /** such as '2011/12/03T10:15:30+01:00' or '2011/12/03T10:15:30.123456789+01:00'. */ + .appendOptional(ISO_OFFSET_DATE_TIME_WITH_SLASH_NS) - /** such as '2011.12.03T10:15:30+01:00' or '2011.12.03T10:15:30.123+01:00'. */ - .appendOptional(ISO_OFFSET_DATE_TIME_WITH_DOT) + /** such as '2011.12.03T10:15:30+01:00' or '2011.12.03T10:15:30.123+01:00'. */ + .appendOptional(ISO_OFFSET_DATE_TIME_WITH_DOT) - /** such as '2011.12.03T10:15:30+01:00' or '2011.12.03T10:15:30.123456+01:00'. */ - .appendOptional(ISO_OFFSET_DATE_TIME_WITH_DOT_US) + /** such as '2011.12.03T10:15:30+01:00' or '2011.12.03T10:15:30.123456+01:00'. */ + .appendOptional(ISO_OFFSET_DATE_TIME_WITH_DOT_US) - /** such as '2011.12.03T10:15:30+01:00' or '2011.12.03T10:15:30.123456789+01:00'. */ - .appendOptional(ISO_OFFSET_DATE_TIME_WITH_DOT_NS) + /** such as '2011.12.03T10:15:30+01:00' or '2011.12.03T10:15:30.123456789+01:00'. */ + .appendOptional(ISO_OFFSET_DATE_TIME_WITH_DOT_NS) - /** such as '2011-12-03 10:15:30+01:00' or '2011-12-03 10:15:30.123+01:00'. */ - .appendOptional(ISO_OFFSET_DATE_TIME_WITH_SPACE) + /** such as '2011-12-03 10:15:30+01:00' or '2011-12-03 10:15:30.123+01:00'. */ + .appendOptional(ISO_OFFSET_DATE_TIME_WITH_SPACE) - /** such as '2011-12-03 10:15:30+01:00' or '2011-12-03 10:15:30.123456+01:00'. */ - .appendOptional(ISO_OFFSET_DATE_TIME_WITH_SPACE_US) + /** such as '2011-12-03 10:15:30+01:00' or '2011-12-03 10:15:30.123456+01:00'. */ + .appendOptional(ISO_OFFSET_DATE_TIME_WITH_SPACE_US) - /** such as '2011-12-03 10:15:30+01:00' or '2011-12-03 10:15:30.123456789+01:00'. */ - .appendOptional(ISO_OFFSET_DATE_TIME_WITH_SPACE_NS) + /** such as '2011-12-03 10:15:30+01:00' or '2011-12-03 10:15:30.123456789+01:00'. */ + .appendOptional(ISO_OFFSET_DATE_TIME_WITH_SPACE_NS) - /** such as '2011/12/03 10:15:30+01:00' or '2011/12/03 10:15:30.123+01:00'. */ - .appendOptional(ISO_OFFSET_DATE_TIME_WITH_SLASH_WITH_SPACE) + /** such as '2011/12/03 10:15:30+01:00' or '2011/12/03 10:15:30.123+01:00'. */ + .appendOptional(ISO_OFFSET_DATE_TIME_WITH_SLASH_WITH_SPACE) - /** such as '2011/12/03 10:15:30+01:00' or '2011/12/03 10:15:30.123456+01:00'. */ - .appendOptional(ISO_OFFSET_DATE_TIME_WITH_SLASH_WITH_SPACE_US) + /** such as '2011/12/03 10:15:30+01:00' or '2011/12/03 10:15:30.123456+01:00'. */ + .appendOptional(ISO_OFFSET_DATE_TIME_WITH_SLASH_WITH_SPACE_US) - /** such as '2011/12/03 10:15:30+01:00' or '2011/12/03 10:15:30.123456789+01:00'. */ - .appendOptional(ISO_OFFSET_DATE_TIME_WITH_SLASH_WITH_SPACE_NS) + /** such as '2011/12/03 10:15:30+01:00' or '2011/12/03 10:15:30.123456789+01:00'. */ + .appendOptional(ISO_OFFSET_DATE_TIME_WITH_SLASH_WITH_SPACE_NS) - /** such as '2011.12.03 10:15:30+01:00' or '2011.12.03 10:15:30.123+01:00'. */ - .appendOptional(ISO_OFFSET_DATE_TIME_WITH_DOT_WITH_SPACE) + /** such as '2011.12.03 10:15:30+01:00' or '2011.12.03 10:15:30.123+01:00'. */ + .appendOptional(ISO_OFFSET_DATE_TIME_WITH_DOT_WITH_SPACE) - /** such as '2011.12.03 10:15:30+01:00' or '2011.12.03 10:15:30.123456+01:00'. */ - .appendOptional(ISO_OFFSET_DATE_TIME_WITH_DOT_WITH_SPACE_US) + /** such as '2011.12.03 10:15:30+01:00' or '2011.12.03 10:15:30.123456+01:00'. */ + .appendOptional(ISO_OFFSET_DATE_TIME_WITH_DOT_WITH_SPACE_US) - /** such as '2011.12.03 10:15:30+01:00' or '2011.12.03 10:15:30.123456789+01:00'. */ - .appendOptional(ISO_OFFSET_DATE_TIME_WITH_DOT_WITH_SPACE_NS) - .toFormatter(); + /** such as '2011.12.03 10:15:30+01:00' or '2011.12.03 10:15:30.123456789+01:00'. */ + .appendOptional(ISO_OFFSET_DATE_TIME_WITH_DOT_WITH_SPACE_NS) + .toFormatter(); public static long convertDatetimeStrToLong(String str, ZoneId zoneId) - throws LogicalOperatorException { + throws LogicalOperatorException { return convertDatetimeStrToLong(str, toZoneOffset(zoneId), 0); } public static long getInstantWithPrecision(String str, String timestampPrecision) - throws LogicalOperatorException { + throws LogicalOperatorException { try { ZonedDateTime zonedDateTime = ZonedDateTime.parse(str, formatter); Instant instant = zonedDateTime.toInstant(); @@ -481,30 +481,30 @@ public static long getInstantWithPrecision(String str, String timestampPrecision /** convert date time string to millisecond, microsecond or nanosecond. */ public static long convertDatetimeStrToLong(String str, ZoneOffset offset, int depth) - throws LogicalOperatorException { + throws LogicalOperatorException { String timestampPrecision = IoTDBDescriptor.getInstance().getConfig().getTimestampPrecision(); if (depth >= 2) { throw new DateTimeException( - String.format( - "Failed to convert %s to millisecond, zone offset is %s, " - + "please input like 2011-12-03T10:15:30 or 2011-12-03T10:15:30+01:00", - str, offset)); + String.format( + "Failed to convert %s to millisecond, zone offset is %s, " + + "please input like 2011-12-03T10:15:30 or 2011-12-03T10:15:30+01:00", + str, offset)); } if (str.contains("Z")) { return convertDatetimeStrToLong(str.substring(0, str.indexOf('Z')) + "+00:00", offset, depth); } else if (str.length() == 10) { return convertDatetimeStrToLong(str + "T00:00:00", offset, depth); } else if (str.length() - str.lastIndexOf('+') != 6 - && str.length() - str.lastIndexOf('-') != 6) { + && str.length() - str.lastIndexOf('-') != 6) { return convertDatetimeStrToLong(str + offset, offset, depth + 1); } else if (str.contains("[") || str.contains("]")) { throw new DateTimeException( - String.format( - "%s with [time-region] at end is not supported now, " - + "please input like 2011-12-03T10:15:30 or 2011-12-03T10:15:30+01:00", - str)); + String.format( + "%s with [time-region] at end is not supported now, " + + "please input like 2011-12-03T10:15:30 or 2011-12-03T10:15:30+01:00", + str)); } return getInstantWithPrecision(str, timestampPrecision); } @@ -527,7 +527,7 @@ public static long convertDurationStrToLong(String duration, String timestampPre public static long convertDurationStrToLong(long currentTime, String duration) { return convertDurationStrToLong( - currentTime, duration, IoTDBDescriptor.getInstance().getConfig().getTimestampPrecision()); + currentTime, duration, IoTDBDescriptor.getInstance().getConfig().getTimestampPrecision()); } /** @@ -537,7 +537,7 @@ public static long convertDurationStrToLong(long currentTime, String duration) { * @return time in milliseconds, microseconds, or nanoseconds depending on the profile */ public static long convertDurationStrToLong( - long currentTime, String duration, String timestampPrecision) { + long currentTime, String duration, String timestampPrecision) { long total = 0; long temp = 0; for (int i = 0; i < duration.length(); i++) { @@ -553,11 +553,11 @@ public static long convertDurationStrToLong( unit += duration.charAt(i); } total += - DatetimeUtils.convertDurationStrToLong( - currentTime == -1 ? -1 : currentTime + total, - temp, - unit.toLowerCase(), - timestampPrecision); + DatetimeUtils.convertDurationStrToLong( + currentTime == -1 ? -1 : currentTime + total, + temp, + unit.toLowerCase(), + timestampPrecision); temp = 0; } } @@ -566,13 +566,13 @@ public static long convertDurationStrToLong( @TestOnly public static long convertDurationStrToLongForTest( - long value, String unit, String timestampPrecision) { + long value, String unit, String timestampPrecision) { return convertDurationStrToLong(-1, value, unit, timestampPrecision); } /** convert duration string to millisecond, microsecond or nanosecond. */ public static long convertDurationStrToLong( - long currentTime, long value, String unit, String timestampPrecision) { + long currentTime, long value, String unit, String timestampPrecision) { DurationUnit durationUnit = DurationUnit.valueOf(unit); long res = value; switch (durationUnit) { From af5a49e4ddcd0507716aeecace34ce5bfe996ff3 Mon Sep 17 00:00:00 2001 From: Yuting Date: Wed, 14 Jul 2021 17:04:55 -0500 Subject: [PATCH 11/28] modify canSkipDelete in StorageGroupProcessor --- .../iotdb/db/engine/storagegroup/StorageGroupProcessor.java | 2 +- .../apache/iotdb/db/engine/storagegroup/TsFileResource.java | 6 +++++- .../db/engine/storagegroup/timeindex/DeviceTimeIndex.java | 1 - 3 files changed, 6 insertions(+), 3 deletions(-) diff --git a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/StorageGroupProcessor.java b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/StorageGroupProcessor.java index 6932095fe44c4..3a1c4a7d4f4b3 100755 --- a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/StorageGroupProcessor.java +++ b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/StorageGroupProcessor.java @@ -1851,7 +1851,7 @@ private boolean canSkipDelete( return false; } - if (tsFileResource.getDevices().contains(deviceId) + if (tsFileResource.isDeviceIdInRes(deviceId) && (deleteEnd >= tsFileResource.getStartTime(deviceId) && deleteStart <= endTime)) { return false; } diff --git a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileResource.java b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileResource.java index 20ffba46faadb..0311a4cf5b25a 100644 --- a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileResource.java +++ b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileResource.java @@ -627,10 +627,14 @@ public boolean stillLives(long timeLowerBound) { return timeIndex.stillLives(timeLowerBound); } + public boolean isDeviceIdInRes(String deviceId) { + return timeIndex.checkDeviceIdExist(deviceId); + } + /** @return true if the device is contained in the TsFile and it lives beyond TTL */ public boolean isSatisfied( String deviceId, Filter timeFilter, boolean isSeq, long ttl, boolean debug) { - if (!timeIndex.checkDeviceIdExist(deviceId)) { + if (!isDeviceIdInRes(deviceId)) { if (debug) { DEBUG_LOGGER.info( "Path: {} file {} is not satisfied because of no device!", deviceId, file); diff --git a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/timeindex/DeviceTimeIndex.java b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/timeindex/DeviceTimeIndex.java index cf8a0ab308f6a..968e963137eeb 100644 --- a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/timeindex/DeviceTimeIndex.java +++ b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/timeindex/DeviceTimeIndex.java @@ -47,7 +47,6 @@ public class DeviceTimeIndex implements ITimeIndex { protected static final Map cachedDevicePool = CachedStringPool.getInstance().getCachedPool(); - private static final Logger DEBUG_LOGGER = LoggerFactory.getLogger("QUERY_DEBUG"); /** start times array. */ protected long[] startTimes; From a7db4d21318c3fa6d229375b1fe9228f93f86444 Mon Sep 17 00:00:00 2001 From: Yuting Date: Wed, 14 Jul 2021 23:54:46 -0500 Subject: [PATCH 12/28] merger master --- .../db/engine/storagegroup/timeindex/DeviceTimeIndex.java | 4 ---- 1 file changed, 4 deletions(-) diff --git a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/timeindex/DeviceTimeIndex.java b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/timeindex/DeviceTimeIndex.java index 968e963137eeb..b93cf8733e6fa 100644 --- a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/timeindex/DeviceTimeIndex.java +++ b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/timeindex/DeviceTimeIndex.java @@ -27,9 +27,6 @@ import org.apache.iotdb.tsfile.utils.RamUsageEstimator; import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; @@ -47,7 +44,6 @@ public class DeviceTimeIndex implements ITimeIndex { protected static final Map cachedDevicePool = CachedStringPool.getInstance().getCachedPool(); - /** start times array. */ protected long[] startTimes; From c32fc8394b4e93b4efdecd2601385c8229f01e9b Mon Sep 17 00:00:00 2001 From: Yuting Date: Thu, 15 Jul 2021 23:02:15 -0500 Subject: [PATCH 13/28] revise isDeciveInRes method name --- .../iotdb/db/engine/storagegroup/StorageGroupProcessor.java | 2 +- .../apache/iotdb/db/engine/storagegroup/TsFileResource.java | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/StorageGroupProcessor.java b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/StorageGroupProcessor.java index 3a1c4a7d4f4b3..be469be2c0a01 100755 --- a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/StorageGroupProcessor.java +++ b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/StorageGroupProcessor.java @@ -1851,7 +1851,7 @@ private boolean canSkipDelete( return false; } - if (tsFileResource.isDeviceIdInRes(deviceId) + if (tsFileResource.isDeviceIdExist(deviceId) && (deleteEnd >= tsFileResource.getStartTime(deviceId) && deleteStart <= endTime)) { return false; } diff --git a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileResource.java b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileResource.java index 0311a4cf5b25a..f0c7d18e4854c 100644 --- a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileResource.java +++ b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileResource.java @@ -627,14 +627,14 @@ public boolean stillLives(long timeLowerBound) { return timeIndex.stillLives(timeLowerBound); } - public boolean isDeviceIdInRes(String deviceId) { + public boolean isDeviceIdExist(String deviceId) { return timeIndex.checkDeviceIdExist(deviceId); } /** @return true if the device is contained in the TsFile and it lives beyond TTL */ public boolean isSatisfied( String deviceId, Filter timeFilter, boolean isSeq, long ttl, boolean debug) { - if (!isDeviceIdInRes(deviceId)) { + if (!timeIndex.checkDeviceIdExist(deviceId)) { if (debug) { DEBUG_LOGGER.info( "Path: {} file {} is not satisfied because of no device!", deviceId, file); From ad364348e11d7f697553171d7b3cf8455b8c9523 Mon Sep 17 00:00:00 2001 From: Yuting Date: Tue, 20 Jul 2021 08:19:05 -0500 Subject: [PATCH 14/28] delete TimeIndexLevel related method in all classes --- .../iotdb/cluster/RemoteTsFileResource.java | 11 ++--- .../resources/conf/iotdb-engine.properties | 5 +-- .../org/apache/iotdb/db/conf/IoTDBConfig.java | 10 ----- .../apache/iotdb/db/conf/IoTDBDescriptor.java | 3 -- .../engine/storagegroup/TsFileResource.java | 13 +++--- .../merge/MaxFileMergeFileSelectorTest.java | 10 ++--- .../IoTDBLoadExternalTsfileIT.java | 42 +++++++------------ 7 files changed, 30 insertions(+), 64 deletions(-) diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/RemoteTsFileResource.java b/cluster/src/main/java/org/apache/iotdb/cluster/RemoteTsFileResource.java index 04fb776a5d66a..6c5f75bf2a91c 100644 --- a/cluster/src/main/java/org/apache/iotdb/cluster/RemoteTsFileResource.java +++ b/cluster/src/main/java/org/apache/iotdb/cluster/RemoteTsFileResource.java @@ -21,8 +21,8 @@ import org.apache.iotdb.cluster.rpc.thrift.Node; import org.apache.iotdb.cluster.utils.NodeSerializeUtils; -import org.apache.iotdb.db.conf.IoTDBDescriptor; import org.apache.iotdb.db.engine.storagegroup.TsFileResource; +import org.apache.iotdb.db.engine.storagegroup.timeindex.DeviceTimeIndex; import org.apache.iotdb.db.utils.SerializeUtils; import java.io.DataOutputStream; @@ -47,7 +47,7 @@ public class RemoteTsFileResource extends TsFileResource { public RemoteTsFileResource() { setClosed(true); - this.timeIndex = IoTDBDescriptor.getInstance().getConfig().getTimeIndexLevel().getTimeIndex(); + this.timeIndex = new DeviceTimeIndex(); } private RemoteTsFileResource(TsFileResource other) throws IOException { @@ -106,12 +106,7 @@ public void deserialize(ByteBuffer buffer) { NodeSerializeUtils.deserialize(source, buffer); setFile(new File(SerializeUtils.deserializeString(buffer))); - timeIndex = - IoTDBDescriptor.getInstance() - .getConfig() - .getTimeIndexLevel() - .getTimeIndex() - .deserialize(buffer); + timeIndex = new DeviceTimeIndex(); withModification = buffer.get() == 1; diff --git a/server/src/assembly/resources/conf/iotdb-engine.properties b/server/src/assembly/resources/conf/iotdb-engine.properties index 9eb19dde5363c..13b2a50a06c23 100644 --- a/server/src/assembly/resources/conf/iotdb-engine.properties +++ b/server/src/assembly/resources/conf/iotdb-engine.properties @@ -279,10 +279,7 @@ timestamp_precision=ms # Datatype: int # virtual_storage_group_num = 1 -# Level of TimeIndex, which records the start time and end time of TsFileResource. Currently, -# DEVICE_TIME_INDEX and FILE_TIME_INDEX are supported, and could not be changed after first set. -# Datatype: TimeIndexLevel -# time_index_level=DEVICE_TIME_INDEX + #################### ### Memory Control Configuration diff --git a/server/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java b/server/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java index 4b8d6b869bea9..7449fe669077e 100644 --- a/server/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java +++ b/server/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java @@ -21,7 +21,6 @@ import org.apache.iotdb.db.conf.directories.DirectoryManager; import org.apache.iotdb.db.engine.compaction.CompactionStrategy; import org.apache.iotdb.db.engine.merge.selector.MergeFileStrategy; -import org.apache.iotdb.db.engine.storagegroup.timeindex.TimeIndexLevel; import org.apache.iotdb.db.exception.LoadConfigurationException; import org.apache.iotdb.db.metadata.MManager; import org.apache.iotdb.db.service.TSServiceImpl; @@ -638,7 +637,6 @@ public class IoTDBConfig { * Level of TimeIndex, which records the start time and end time of TsFileResource. Currently, * DEVICE_TIME_INDEX and FILE_TIME_INDEX are supported, and could not be changed after first set. */ - private TimeIndexLevel timeIndexLevel = TimeIndexLevel.DEVICE_TIME_INDEX; // just for test // wait for 60 second by default. @@ -812,14 +810,6 @@ public void setPartitionInterval(long partitionInterval) { this.partitionInterval = partitionInterval; } - public TimeIndexLevel getTimeIndexLevel() { - return timeIndexLevel; - } - - public void setTimeIndexLevel(String timeIndexLevel) { - this.timeIndexLevel = TimeIndexLevel.valueOf(timeIndexLevel); - } - void updatePath() { formulateFolders(); confirmMultiDirStrategy(); diff --git a/server/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java b/server/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java index a3f271afca815..2a2bbcab9ba55 100644 --- a/server/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java +++ b/server/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java @@ -604,9 +604,6 @@ private void loadProps() { Long.parseLong( properties.getProperty("default_ttl", String.valueOf(conf.getDefaultTTL())))); - conf.setTimeIndexLevel( - properties.getProperty("time_index_level", String.valueOf(conf.getTimeIndexLevel()))); - // the default fill interval in LinearFill and PreviousFill conf.setDefaultFillInterval( Integer.parseInt( diff --git a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileResource.java b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileResource.java index f0c7d18e4854c..5d77d616a31be 100644 --- a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileResource.java +++ b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileResource.java @@ -25,7 +25,6 @@ import org.apache.iotdb.db.engine.storagegroup.StorageGroupProcessor.UpgradeTsFileResourceCallBack; import org.apache.iotdb.db.engine.storagegroup.timeindex.DeviceTimeIndex; import org.apache.iotdb.db.engine.storagegroup.timeindex.ITimeIndex; -import org.apache.iotdb.db.engine.storagegroup.timeindex.TimeIndexLevel; import org.apache.iotdb.db.engine.upgrade.UpgradeTask; import org.apache.iotdb.db.exception.PartitionViolationException; import org.apache.iotdb.db.service.UpgradeSevice; @@ -179,16 +178,16 @@ public TsFileResource(TsFileResource other) throws IOException { public TsFileResource(File file) { this.file = file; this.version = FilePathUtils.splitAndGetTsFileVersion(this.file.getName()); - this.timeIndex = config.getTimeIndexLevel().getTimeIndex(); - this.timeIndexType = (byte) config.getTimeIndexLevel().ordinal(); + this.timeIndex = new DeviceTimeIndex(); // config.getTimeIndexLevel().getTimeIndex(); + this.timeIndexType = 1; } /** unsealed TsFile */ public TsFileResource(File file, TsFileProcessor processor) { this.file = file; this.version = FilePathUtils.splitAndGetTsFileVersion(this.file.getName()); - this.timeIndex = config.getTimeIndexLevel().getTimeIndex(); - this.timeIndexType = (byte) config.getTimeIndexLevel().ordinal(); + this.timeIndex = new DeviceTimeIndex(); + this.timeIndexType = 1; this.processor = processor; } @@ -349,7 +348,9 @@ public void deserialize() throws IOException { try (InputStream inputStream = fsFactory.getBufferedInputStream(file + RESOURCE_SUFFIX)) { readVersionNumber(inputStream); timeIndexType = ReadWriteIOUtils.readBytes(inputStream, 1)[0]; - timeIndex = TimeIndexLevel.valueOf(timeIndexType).getTimeIndex().deserialize(inputStream); + // timeIndex = + // TimeIndexLevel.valueOf(timeIndexType).getTimeIndex().deserialize(inputStream); + timeIndex = new DeviceTimeIndex().deserialize(inputStream); maxPlanIndex = ReadWriteIOUtils.readLong(inputStream); minPlanIndex = ReadWriteIOUtils.readLong(inputStream); if (inputStream.available() > 0) { diff --git a/server/src/test/java/org/apache/iotdb/db/engine/merge/MaxFileMergeFileSelectorTest.java b/server/src/test/java/org/apache/iotdb/db/engine/merge/MaxFileMergeFileSelectorTest.java index f852a88122309..13e9eb3474cf9 100644 --- a/server/src/test/java/org/apache/iotdb/db/engine/merge/MaxFileMergeFileSelectorTest.java +++ b/server/src/test/java/org/apache/iotdb/db/engine/merge/MaxFileMergeFileSelectorTest.java @@ -20,12 +20,12 @@ package org.apache.iotdb.db.engine.merge; import org.apache.iotdb.db.conf.IoTDBConstant; -import org.apache.iotdb.db.conf.IoTDBDescriptor; import org.apache.iotdb.db.constant.TestConstant; 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.storagegroup.TsFileResource; +import org.apache.iotdb.db.engine.storagegroup.timeindex.DeviceTimeIndex; import org.apache.iotdb.db.engine.storagegroup.timeindex.ITimeIndex; import org.apache.iotdb.db.exception.MergeException; import org.apache.iotdb.tsfile.exception.write.WriteProcessException; @@ -131,8 +131,8 @@ public void testFileOpenSelection() Field timeIndexField = TsFileResource.class.getDeclaredField("timeIndex"); timeIndexField.setAccessible(true); ITimeIndex timeIndex = (ITimeIndex) timeIndexField.get(secondTsFileResource); - ITimeIndex newTimeIndex = - IoTDBDescriptor.getInstance().getConfig().getTimeIndexLevel().getTimeIndex(); + ITimeIndex newTimeIndex = new DeviceTimeIndex(); + for (String device : devices) { newTimeIndex.updateStartTime(device, timeIndex.getStartTime(device)); } @@ -183,8 +183,8 @@ public void testFileOpenSelectionFromCompaction() Field timeIndexField = TsFileResource.class.getDeclaredField("timeIndex"); timeIndexField.setAccessible(true); ITimeIndex timeIndex = (ITimeIndex) timeIndexField.get(secondTsFileResource); - ITimeIndex newTimeIndex = - IoTDBDescriptor.getInstance().getConfig().getTimeIndexLevel().getTimeIndex(); + ITimeIndex newTimeIndex = new DeviceTimeIndex(); + for (String device : devices) { newTimeIndex.updateStartTime(device, timeIndex.getStartTime(device)); } diff --git a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBLoadExternalTsfileIT.java b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBLoadExternalTsfileIT.java index f1a149f155600..4c484813269cd 100644 --- a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBLoadExternalTsfileIT.java +++ b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBLoadExternalTsfileIT.java @@ -23,7 +23,6 @@ import org.apache.iotdb.db.engine.StorageEngine; import org.apache.iotdb.db.engine.compaction.CompactionStrategy; import org.apache.iotdb.db.engine.storagegroup.TsFileResource; -import org.apache.iotdb.db.engine.storagegroup.timeindex.TimeIndexLevel; import org.apache.iotdb.db.engine.storagegroup.virtualSg.HashVirtualPartitioner; import org.apache.iotdb.db.exception.StorageEngineException; import org.apache.iotdb.db.exception.metadata.IllegalPathException; @@ -457,33 +456,20 @@ public void loadUnsequenceTsfileTest() throws SQLException { .getProcessor(new PartialPath("root.vehicle")) .getUnSequenceFileList() .size()); - if (config.getTimeIndexLevel().equals(TimeIndexLevel.DEVICE_TIME_INDEX)) { - assertEquals( - 1, - StorageEngine.getInstance() - .getProcessor(new PartialPath("root.test")) - .getUnSequenceFileList() - .size()); - assertEquals( - 3, - StorageEngine.getInstance() - .getProcessor(new PartialPath("root.test")) - .getSequenceFileTreeSet() - .size()); - } else if (config.getTimeIndexLevel().equals(TimeIndexLevel.FILE_TIME_INDEX)) { - assertEquals( - 2, - StorageEngine.getInstance() - .getProcessor(new PartialPath("root.test")) - .getUnSequenceFileList() - .size()); - assertEquals( - 2, - StorageEngine.getInstance() - .getProcessor(new PartialPath("root.test")) - .getSequenceFileTreeSet() - .size()); - } + + assertEquals( + 1, + StorageEngine.getInstance() + .getProcessor(new PartialPath("root.test")) + .getUnSequenceFileList() + .size()); + assertEquals( + 3, + StorageEngine.getInstance() + .getProcessor(new PartialPath("root.test")) + .getSequenceFileTreeSet() + .size()); + assertNotNull(tmpDir.listFiles()); assertEquals( 0, From 9ee239213093aa59930ab0595d67f6edd714f8d0 Mon Sep 17 00:00:00 2001 From: Yuting Date: Tue, 20 Jul 2021 15:28:15 -0500 Subject: [PATCH 15/28] remove comment --- .../apache/iotdb/db/engine/storagegroup/TsFileResource.java | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileResource.java b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileResource.java index 5d77d616a31be..b11779f71d2d0 100644 --- a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileResource.java +++ b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileResource.java @@ -178,7 +178,7 @@ public TsFileResource(TsFileResource other) throws IOException { public TsFileResource(File file) { this.file = file; this.version = FilePathUtils.splitAndGetTsFileVersion(this.file.getName()); - this.timeIndex = new DeviceTimeIndex(); // config.getTimeIndexLevel().getTimeIndex(); + this.timeIndex = new DeviceTimeIndex(); this.timeIndexType = 1; } @@ -348,8 +348,6 @@ public void deserialize() throws IOException { try (InputStream inputStream = fsFactory.getBufferedInputStream(file + RESOURCE_SUFFIX)) { readVersionNumber(inputStream); timeIndexType = ReadWriteIOUtils.readBytes(inputStream, 1)[0]; - // timeIndex = - // TimeIndexLevel.valueOf(timeIndexType).getTimeIndex().deserialize(inputStream); timeIndex = new DeviceTimeIndex().deserialize(inputStream); maxPlanIndex = ReadWriteIOUtils.readLong(inputStream); minPlanIndex = ReadWriteIOUtils.readLong(inputStream); From 79d38d69e9cac1fe9e06a57aa7b99ea2552d088a Mon Sep 17 00:00:00 2001 From: Yuting Date: Tue, 20 Jul 2021 21:10:31 -0500 Subject: [PATCH 16/28] delete TimeIndexLevel class --- .../timeindex/TimeIndexLevel.java | 45 ------------------- 1 file changed, 45 deletions(-) delete mode 100644 server/src/main/java/org/apache/iotdb/db/engine/storagegroup/timeindex/TimeIndexLevel.java diff --git a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/timeindex/TimeIndexLevel.java b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/timeindex/TimeIndexLevel.java deleted file mode 100644 index 95e9b9eee8fc5..0000000000000 --- a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/timeindex/TimeIndexLevel.java +++ /dev/null @@ -1,45 +0,0 @@ -/* - * 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.timeindex; - -public enum TimeIndexLevel { - /** file to time index (small memory foot print) */ - FILE_TIME_INDEX, - - /** device to time index (large memory foot print) */ - DEVICE_TIME_INDEX; - - public ITimeIndex getTimeIndex() { - switch (this) { - case FILE_TIME_INDEX: - return new FileTimeIndex(); - case DEVICE_TIME_INDEX: - default: - return new DeviceTimeIndex(); - } - } - - public static TimeIndexLevel valueOf(int ordinal) { - if (ordinal < 0 || ordinal >= values().length) { - throw new IndexOutOfBoundsException("Invalid ordinal"); - } - return values()[ordinal]; - } -} From fc1434140db715e939b4c6d8b4f437ace07cd463 Mon Sep 17 00:00:00 2001 From: Yuting Date: Tue, 20 Jul 2021 22:10:58 -0500 Subject: [PATCH 17/28] add TimeIndexLevel class --- .../timeindex/TimeIndexLevel.java | 45 +++++++++++++++++++ 1 file changed, 45 insertions(+) create mode 100644 server/src/main/java/org/apache/iotdb/db/engine/storagegroup/timeindex/TimeIndexLevel.java diff --git a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/timeindex/TimeIndexLevel.java b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/timeindex/TimeIndexLevel.java new file mode 100644 index 0000000000000..95e9b9eee8fc5 --- /dev/null +++ b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/timeindex/TimeIndexLevel.java @@ -0,0 +1,45 @@ +/* + * 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.timeindex; + +public enum TimeIndexLevel { + /** file to time index (small memory foot print) */ + FILE_TIME_INDEX, + + /** device to time index (large memory foot print) */ + DEVICE_TIME_INDEX; + + public ITimeIndex getTimeIndex() { + switch (this) { + case FILE_TIME_INDEX: + return new FileTimeIndex(); + case DEVICE_TIME_INDEX: + default: + return new DeviceTimeIndex(); + } + } + + public static TimeIndexLevel valueOf(int ordinal) { + if (ordinal < 0 || ordinal >= values().length) { + throw new IndexOutOfBoundsException("Invalid ordinal"); + } + return values()[ordinal]; + } +} From e21590d81b6876ad213c677c38f3e905a6059aa8 Mon Sep 17 00:00:00 2001 From: Yuting Date: Fri, 23 Jul 2021 11:35:57 -0500 Subject: [PATCH 18/28] recover TimeIndexLevel and modify getDevices method in FileTimeIndex --- .../iotdb/cluster/RemoteTsFileResource.java | 11 ++++- .../resources/conf/iotdb-engine.properties | 5 ++- .../org/apache/iotdb/db/conf/IoTDBConfig.java | 11 ++++- .../apache/iotdb/db/conf/IoTDBDescriptor.java | 3 ++ .../db/engine/memtable/AbstractMemTable.java | 14 ++++--- .../iotdb/db/engine/memtable/IMemTable.java | 5 +++ .../engine/storagegroup/TsFileProcessor.java | 30 +++++++++++--- .../engine/storagegroup/TsFileResource.java | 22 +++++----- .../timeindex/DeviceTimeIndex.java | 2 +- .../storagegroup/timeindex/FileTimeIndex.java | 18 ++++++++- .../storagegroup/timeindex/ITimeIndex.java | 2 +- .../merge/MaxFileMergeFileSelectorTest.java | 5 ++- .../IoTDBLoadExternalTsfileIT.java | 40 +++++++++++++------ 13 files changed, 123 insertions(+), 45 deletions(-) diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/RemoteTsFileResource.java b/cluster/src/main/java/org/apache/iotdb/cluster/RemoteTsFileResource.java index 6c5f75bf2a91c..2dea75e3b3323 100644 --- a/cluster/src/main/java/org/apache/iotdb/cluster/RemoteTsFileResource.java +++ b/cluster/src/main/java/org/apache/iotdb/cluster/RemoteTsFileResource.java @@ -21,6 +21,7 @@ import org.apache.iotdb.cluster.rpc.thrift.Node; import org.apache.iotdb.cluster.utils.NodeSerializeUtils; +import org.apache.iotdb.db.conf.IoTDBDescriptor; import org.apache.iotdb.db.engine.storagegroup.TsFileResource; import org.apache.iotdb.db.engine.storagegroup.timeindex.DeviceTimeIndex; import org.apache.iotdb.db.utils.SerializeUtils; @@ -47,7 +48,8 @@ public class RemoteTsFileResource extends TsFileResource { public RemoteTsFileResource() { setClosed(true); - this.timeIndex = new DeviceTimeIndex(); + this.timeIndex = + IoTDBDescriptor.getInstance().getConfig().getTimeIndexLevel().getTimeIndex(); } private RemoteTsFileResource(TsFileResource other) throws IOException { @@ -106,7 +108,12 @@ public void deserialize(ByteBuffer buffer) { NodeSerializeUtils.deserialize(source, buffer); setFile(new File(SerializeUtils.deserializeString(buffer))); - timeIndex = new DeviceTimeIndex(); + timeIndex = + IoTDBDescriptor.getInstance() + .getConfig() + .getTimeIndexLevel() + .getTimeIndex() + .deserialize(buffer); withModification = buffer.get() == 1; diff --git a/server/src/assembly/resources/conf/iotdb-engine.properties b/server/src/assembly/resources/conf/iotdb-engine.properties index 13b2a50a06c23..9eb19dde5363c 100644 --- a/server/src/assembly/resources/conf/iotdb-engine.properties +++ b/server/src/assembly/resources/conf/iotdb-engine.properties @@ -279,7 +279,10 @@ timestamp_precision=ms # Datatype: int # virtual_storage_group_num = 1 - +# Level of TimeIndex, which records the start time and end time of TsFileResource. Currently, +# DEVICE_TIME_INDEX and FILE_TIME_INDEX are supported, and could not be changed after first set. +# Datatype: TimeIndexLevel +# time_index_level=DEVICE_TIME_INDEX #################### ### Memory Control Configuration diff --git a/server/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java b/server/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java index 7449fe669077e..bf395a96c5fef 100644 --- a/server/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java +++ b/server/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java @@ -21,6 +21,7 @@ import org.apache.iotdb.db.conf.directories.DirectoryManager; import org.apache.iotdb.db.engine.compaction.CompactionStrategy; import org.apache.iotdb.db.engine.merge.selector.MergeFileStrategy; +import org.apache.iotdb.db.engine.storagegroup.timeindex.TimeIndexLevel; import org.apache.iotdb.db.exception.LoadConfigurationException; import org.apache.iotdb.db.metadata.MManager; import org.apache.iotdb.db.service.TSServiceImpl; @@ -637,7 +638,7 @@ public class IoTDBConfig { * Level of TimeIndex, which records the start time and end time of TsFileResource. Currently, * DEVICE_TIME_INDEX and FILE_TIME_INDEX are supported, and could not be changed after first set. */ - + private TimeIndexLevel timeIndexLevel = TimeIndexLevel.DEVICE_TIME_INDEX; // just for test // wait for 60 second by default. private int thriftServerAwaitTimeForStopService = 60; @@ -810,6 +811,14 @@ public void setPartitionInterval(long partitionInterval) { this.partitionInterval = partitionInterval; } + public TimeIndexLevel getTimeIndexLevel() { + return timeIndexLevel; + } + + public void setTimeIndexLevel(String timeIndexLevel) { + this.timeIndexLevel = TimeIndexLevel.valueOf(timeIndexLevel); + } + void updatePath() { formulateFolders(); confirmMultiDirStrategy(); diff --git a/server/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java b/server/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java index 2a2bbcab9ba55..67e3ac52ef3ea 100644 --- a/server/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java +++ b/server/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java @@ -604,6 +604,9 @@ private void loadProps() { Long.parseLong( properties.getProperty("default_ttl", String.valueOf(conf.getDefaultTTL())))); + conf.setTimeIndexLevel( + properties.getProperty("time_index_level", String.valueOf(conf.getTimeIndexLevel()))); + // the default fill interval in LinearFill and PreviousFill conf.setDefaultFillInterval( Integer.parseInt( diff --git a/server/src/main/java/org/apache/iotdb/db/engine/memtable/AbstractMemTable.java b/server/src/main/java/org/apache/iotdb/db/engine/memtable/AbstractMemTable.java index 0acb9e11e7f08..2ae01e7579da5 100644 --- a/server/src/main/java/org/apache/iotdb/db/engine/memtable/AbstractMemTable.java +++ b/server/src/main/java/org/apache/iotdb/db/engine/memtable/AbstractMemTable.java @@ -37,11 +37,7 @@ import org.apache.iotdb.tsfile.write.schema.VectorMeasurementSchema; import java.io.IOException; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.Iterator; -import java.util.List; -import java.util.Map; +import java.util.*; import java.util.Map.Entry; public abstract class AbstractMemTable implements IMemTable { @@ -436,4 +432,12 @@ void updatePlanIndexes(long index) { maxPlanIndex = Math.max(index, maxPlanIndex); minPlanIndex = Math.min(index, minPlanIndex); } + + @Override + public Set getDevices() { + if (memTableMap != null) { + return memTableMap.keySet(); + } + return Collections.emptySet(); + } } diff --git a/server/src/main/java/org/apache/iotdb/db/engine/memtable/IMemTable.java b/server/src/main/java/org/apache/iotdb/db/engine/memtable/IMemTable.java index e5c9e32a9c7d0..5c66b003c33c8 100644 --- a/server/src/main/java/org/apache/iotdb/db/engine/memtable/IMemTable.java +++ b/server/src/main/java/org/apache/iotdb/db/engine/memtable/IMemTable.java @@ -31,6 +31,7 @@ import java.io.IOException; import java.util.List; import java.util.Map; +import java.util.Set; /** * IMemTable is designed to store data points which are not flushed into TsFile yet. An instance of @@ -140,4 +141,8 @@ ReadOnlyMemChunk query( long getMaxPlanIndex(); long getMinPlanIndex(); + + + /** get devices from memTable * */ + Set getDevices(); } diff --git a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileProcessor.java b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileProcessor.java index ff19c8991b1ab..3d6501d13589a 100644 --- a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileProcessor.java +++ b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileProcessor.java @@ -72,11 +72,7 @@ import java.io.File; import java.io.IOException; -import java.util.ArrayList; -import java.util.Collection; -import java.util.Iterator; -import java.util.List; -import java.util.Set; +import java.util.*; import java.util.concurrent.ConcurrentLinkedDeque; import java.util.concurrent.locks.ReadWriteLock; import java.util.concurrent.locks.ReentrantReadWriteLock; @@ -1330,4 +1326,28 @@ public void submitAFlushTask() { public boolean alreadyMarkedClosing() { return shouldClose; } + + /** + * get all devices when the TsFile is not close. It consists of three part: 1. flushingMemTables, + * workMemtable is added into the flushingMemTables and memtables are ready to write to disk 2. + * workMemTable, the memtables ready to be written in memory 3. writer, the memtables which have + * been written to disk + * + * @return a set of all deviceId + */ + public Set getDevices() { + Set devicesSet = new HashSet<>(); + + for (IMemTable item : flushingMemTables) { + devicesSet.addAll(item.getDevices()); + } + if (workMemTable != null) { + devicesSet.addAll(workMemTable.getDevices()); + } + + if (writer != null) { + devicesSet.addAll(writer.getMetadatasForQuery().keySet()); + } + return devicesSet; + } } diff --git a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileResource.java b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileResource.java index b11779f71d2d0..e7689882d2543 100644 --- a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileResource.java +++ b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileResource.java @@ -25,6 +25,7 @@ import org.apache.iotdb.db.engine.storagegroup.StorageGroupProcessor.UpgradeTsFileResourceCallBack; import org.apache.iotdb.db.engine.storagegroup.timeindex.DeviceTimeIndex; import org.apache.iotdb.db.engine.storagegroup.timeindex.ITimeIndex; +import org.apache.iotdb.db.engine.storagegroup.timeindex.TimeIndexLevel; import org.apache.iotdb.db.engine.upgrade.UpgradeTask; import org.apache.iotdb.db.exception.PartitionViolationException; import org.apache.iotdb.db.service.UpgradeSevice; @@ -40,6 +41,7 @@ import org.apache.iotdb.tsfile.file.metadata.statistics.Statistics; import org.apache.iotdb.tsfile.fileSystem.FSFactoryProducer; import org.apache.iotdb.tsfile.fileSystem.fsFactory.FSFactory; +import org.apache.iotdb.tsfile.read.TsFileSequenceReader; import org.apache.iotdb.tsfile.read.filter.basic.Filter; import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils; @@ -53,13 +55,7 @@ import java.nio.file.FileAlreadyExistsException; import java.nio.file.Files; import java.nio.file.Paths; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Objects; -import java.util.Random; -import java.util.Set; +import java.util.*; import static org.apache.iotdb.db.conf.IoTDBConstant.FILE_NAME_SEPARATOR; import static org.apache.iotdb.db.conf.IoTDBConstant.FILE_NAME_SUFFIX_INDEX; @@ -178,16 +174,16 @@ public TsFileResource(TsFileResource other) throws IOException { public TsFileResource(File file) { this.file = file; this.version = FilePathUtils.splitAndGetTsFileVersion(this.file.getName()); - this.timeIndex = new DeviceTimeIndex(); - this.timeIndexType = 1; + this.timeIndex = config.getTimeIndexLevel().getTimeIndex(); + this.timeIndexType = (byte) config.getTimeIndexLevel().ordinal(); } /** unsealed TsFile */ public TsFileResource(File file, TsFileProcessor processor) { this.file = file; this.version = FilePathUtils.splitAndGetTsFileVersion(this.file.getName()); - this.timeIndex = new DeviceTimeIndex(); - this.timeIndexType = 1; + this.timeIndex = config.getTimeIndexLevel().getTimeIndex(); + this.timeIndexType = (byte) config.getTimeIndexLevel().ordinal(); this.processor = processor; } @@ -348,7 +344,7 @@ public void deserialize() throws IOException { try (InputStream inputStream = fsFactory.getBufferedInputStream(file + RESOURCE_SUFFIX)) { readVersionNumber(inputStream); timeIndexType = ReadWriteIOUtils.readBytes(inputStream, 1)[0]; - timeIndex = new DeviceTimeIndex().deserialize(inputStream); + timeIndex = TimeIndexLevel.valueOf(timeIndexType).getTimeIndex().deserialize(inputStream); maxPlanIndex = ReadWriteIOUtils.readLong(inputStream); minPlanIndex = ReadWriteIOUtils.readLong(inputStream); if (inputStream.available() > 0) { @@ -468,7 +464,7 @@ public long getEndTime(String deviceId) { } public Set getDevices() { - return timeIndex.getDevices(); + return timeIndex.getDevices(file.getPath()); } public boolean endTimeEmpty() { diff --git a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/timeindex/DeviceTimeIndex.java b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/timeindex/DeviceTimeIndex.java index b93cf8733e6fa..013f1114349ec 100644 --- a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/timeindex/DeviceTimeIndex.java +++ b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/timeindex/DeviceTimeIndex.java @@ -146,7 +146,7 @@ public void close() { } @Override - public Set getDevices() { + public Set getDevices(String tsFilePath) { return deviceToIndex.keySet(); } diff --git a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/timeindex/FileTimeIndex.java b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/timeindex/FileTimeIndex.java index 9cc46ebcb6770..acc15d313d9db 100644 --- a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/timeindex/FileTimeIndex.java +++ b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/timeindex/FileTimeIndex.java @@ -21,24 +21,30 @@ import org.apache.iotdb.db.engine.StorageEngine; import org.apache.iotdb.db.exception.PartitionViolationException; +import org.apache.iotdb.db.query.control.FileReaderManager; import org.apache.iotdb.db.rescon.CachedStringPool; import org.apache.iotdb.db.utils.FilePathUtils; import org.apache.iotdb.db.utils.SerializeUtils; +import org.apache.iotdb.tsfile.read.TsFileSequenceReader; import org.apache.iotdb.tsfile.utils.RamUsageEstimator; import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils; import io.netty.util.internal.ConcurrentSet; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; import java.nio.ByteBuffer; +import java.util.Collections; import java.util.HashSet; import java.util.Map; import java.util.Set; public class FileTimeIndex implements ITimeIndex { + private static final Logger logger = LoggerFactory.getLogger(FileTimeIndex.class); protected static final Map cachedDevicePool = CachedStringPool.getInstance().getCachedPool(); @@ -114,8 +120,16 @@ public void close() { } @Override - public Set getDevices() { - return devices; + public Set getDevices(String tsFilePath) { + try { + TsFileSequenceReader fileReader = + FileReaderManager.getInstance().get(tsFilePath, true); + return new HashSet<>(fileReader.getAllDevices()); + } catch (IOException e) { + logger.error("Can't read file {} from disk ", tsFilePath, e); + } + return Collections.emptySet(); + // return devices; } @Override diff --git a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/timeindex/ITimeIndex.java b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/timeindex/ITimeIndex.java index 37103629b2f8a..d53dd48515152 100644 --- a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/timeindex/ITimeIndex.java +++ b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/timeindex/ITimeIndex.java @@ -62,7 +62,7 @@ public interface ITimeIndex { * * @return device names */ - Set getDevices(); + Set getDevices(String tsFilePath); /** @return whether end time is empty (Long.MIN_VALUE) */ boolean endTimeEmpty(); diff --git a/server/src/test/java/org/apache/iotdb/db/engine/merge/MaxFileMergeFileSelectorTest.java b/server/src/test/java/org/apache/iotdb/db/engine/merge/MaxFileMergeFileSelectorTest.java index 13e9eb3474cf9..4f5dadbe057ca 100644 --- a/server/src/test/java/org/apache/iotdb/db/engine/merge/MaxFileMergeFileSelectorTest.java +++ b/server/src/test/java/org/apache/iotdb/db/engine/merge/MaxFileMergeFileSelectorTest.java @@ -20,6 +20,7 @@ package org.apache.iotdb.db.engine.merge; import org.apache.iotdb.db.conf.IoTDBConstant; +import org.apache.iotdb.db.conf.IoTDBDescriptor; import org.apache.iotdb.db.constant.TestConstant; import org.apache.iotdb.db.engine.merge.manage.MergeResource; import org.apache.iotdb.db.engine.merge.selector.IMergeFileSelector; @@ -131,7 +132,7 @@ public void testFileOpenSelection() Field timeIndexField = TsFileResource.class.getDeclaredField("timeIndex"); timeIndexField.setAccessible(true); ITimeIndex timeIndex = (ITimeIndex) timeIndexField.get(secondTsFileResource); - ITimeIndex newTimeIndex = new DeviceTimeIndex(); + ITimeIndex newTimeIndex = IoTDBDescriptor.getInstance().getConfig().getTimeIndexLevel().getTimeIndex(); for (String device : devices) { newTimeIndex.updateStartTime(device, timeIndex.getStartTime(device)); @@ -183,7 +184,7 @@ public void testFileOpenSelectionFromCompaction() Field timeIndexField = TsFileResource.class.getDeclaredField("timeIndex"); timeIndexField.setAccessible(true); ITimeIndex timeIndex = (ITimeIndex) timeIndexField.get(secondTsFileResource); - ITimeIndex newTimeIndex = new DeviceTimeIndex(); + ITimeIndex newTimeIndex = IoTDBDescriptor.getInstance().getConfig().getTimeIndexLevel().getTimeIndex(); for (String device : devices) { newTimeIndex.updateStartTime(device, timeIndex.getStartTime(device)); diff --git a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBLoadExternalTsfileIT.java b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBLoadExternalTsfileIT.java index 4c484813269cd..7caffe4f7c198 100644 --- a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBLoadExternalTsfileIT.java +++ b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBLoadExternalTsfileIT.java @@ -23,6 +23,7 @@ import org.apache.iotdb.db.engine.StorageEngine; import org.apache.iotdb.db.engine.compaction.CompactionStrategy; import org.apache.iotdb.db.engine.storagegroup.TsFileResource; +import org.apache.iotdb.db.engine.storagegroup.timeindex.TimeIndexLevel; import org.apache.iotdb.db.engine.storagegroup.virtualSg.HashVirtualPartitioner; import org.apache.iotdb.db.exception.StorageEngineException; import org.apache.iotdb.db.exception.metadata.IllegalPathException; @@ -457,18 +458,33 @@ public void loadUnsequenceTsfileTest() throws SQLException { .getUnSequenceFileList() .size()); - assertEquals( - 1, - StorageEngine.getInstance() - .getProcessor(new PartialPath("root.test")) - .getUnSequenceFileList() - .size()); - assertEquals( - 3, - StorageEngine.getInstance() - .getProcessor(new PartialPath("root.test")) - .getSequenceFileTreeSet() - .size()); + if (config.getTimeIndexLevel().equals(TimeIndexLevel.DEVICE_TIME_INDEX)) { + assertEquals( + 1, + StorageEngine.getInstance() + .getProcessor(new PartialPath("root.test")) + .getUnSequenceFileList() + .size()); + assertEquals( + 3, + StorageEngine.getInstance() + .getProcessor(new PartialPath("root.test")) + .getSequenceFileTreeSet() + .size()); + } else if (config.getTimeIndexLevel().equals(TimeIndexLevel.FILE_TIME_INDEX)) { + assertEquals( + 2, + StorageEngine.getInstance() + .getProcessor(new PartialPath("root.test")) + .getUnSequenceFileList() + .size()); + assertEquals( + 2, + StorageEngine.getInstance() + .getProcessor(new PartialPath("root.test")) + .getSequenceFileTreeSet() + .size()); + } assertNotNull(tmpDir.listFiles()); assertEquals( From d77555c477f9e0388393d194c87fdea221c38f26 Mon Sep 17 00:00:00 2001 From: Yuting Date: Fri, 23 Jul 2021 11:40:15 -0500 Subject: [PATCH 19/28] delete getDevices in IMemTable --- .../iotdb/cluster/RemoteTsFileResource.java | 14 +++---- .../apache/iotdb/db/conf/IoTDBDescriptor.java | 2 +- .../db/engine/memtable/AbstractMemTable.java | 8 ---- .../iotdb/db/engine/memtable/IMemTable.java | 5 --- .../engine/storagegroup/TsFileProcessor.java | 24 ----------- .../engine/storagegroup/TsFileResource.java | 3 +- .../storagegroup/timeindex/FileTimeIndex.java | 3 +- .../merge/MaxFileMergeFileSelectorTest.java | 7 ++-- .../IoTDBLoadExternalTsfileIT.java | 40 +++++++++---------- 9 files changed, 33 insertions(+), 73 deletions(-) diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/RemoteTsFileResource.java b/cluster/src/main/java/org/apache/iotdb/cluster/RemoteTsFileResource.java index 2dea75e3b3323..04fb776a5d66a 100644 --- a/cluster/src/main/java/org/apache/iotdb/cluster/RemoteTsFileResource.java +++ b/cluster/src/main/java/org/apache/iotdb/cluster/RemoteTsFileResource.java @@ -23,7 +23,6 @@ import org.apache.iotdb.cluster.utils.NodeSerializeUtils; import org.apache.iotdb.db.conf.IoTDBDescriptor; import org.apache.iotdb.db.engine.storagegroup.TsFileResource; -import org.apache.iotdb.db.engine.storagegroup.timeindex.DeviceTimeIndex; import org.apache.iotdb.db.utils.SerializeUtils; import java.io.DataOutputStream; @@ -48,8 +47,7 @@ public class RemoteTsFileResource extends TsFileResource { public RemoteTsFileResource() { setClosed(true); - this.timeIndex = - IoTDBDescriptor.getInstance().getConfig().getTimeIndexLevel().getTimeIndex(); + this.timeIndex = IoTDBDescriptor.getInstance().getConfig().getTimeIndexLevel().getTimeIndex(); } private RemoteTsFileResource(TsFileResource other) throws IOException { @@ -109,11 +107,11 @@ public void deserialize(ByteBuffer buffer) { setFile(new File(SerializeUtils.deserializeString(buffer))); timeIndex = - IoTDBDescriptor.getInstance() - .getConfig() - .getTimeIndexLevel() - .getTimeIndex() - .deserialize(buffer); + IoTDBDescriptor.getInstance() + .getConfig() + .getTimeIndexLevel() + .getTimeIndex() + .deserialize(buffer); withModification = buffer.get() == 1; diff --git a/server/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java b/server/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java index 67e3ac52ef3ea..a3f271afca815 100644 --- a/server/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java +++ b/server/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java @@ -605,7 +605,7 @@ private void loadProps() { properties.getProperty("default_ttl", String.valueOf(conf.getDefaultTTL())))); conf.setTimeIndexLevel( - properties.getProperty("time_index_level", String.valueOf(conf.getTimeIndexLevel()))); + properties.getProperty("time_index_level", String.valueOf(conf.getTimeIndexLevel()))); // the default fill interval in LinearFill and PreviousFill conf.setDefaultFillInterval( diff --git a/server/src/main/java/org/apache/iotdb/db/engine/memtable/AbstractMemTable.java b/server/src/main/java/org/apache/iotdb/db/engine/memtable/AbstractMemTable.java index 2ae01e7579da5..32683c6f69333 100644 --- a/server/src/main/java/org/apache/iotdb/db/engine/memtable/AbstractMemTable.java +++ b/server/src/main/java/org/apache/iotdb/db/engine/memtable/AbstractMemTable.java @@ -432,12 +432,4 @@ void updatePlanIndexes(long index) { maxPlanIndex = Math.max(index, maxPlanIndex); minPlanIndex = Math.min(index, minPlanIndex); } - - @Override - public Set getDevices() { - if (memTableMap != null) { - return memTableMap.keySet(); - } - return Collections.emptySet(); - } } diff --git a/server/src/main/java/org/apache/iotdb/db/engine/memtable/IMemTable.java b/server/src/main/java/org/apache/iotdb/db/engine/memtable/IMemTable.java index 5c66b003c33c8..e5c9e32a9c7d0 100644 --- a/server/src/main/java/org/apache/iotdb/db/engine/memtable/IMemTable.java +++ b/server/src/main/java/org/apache/iotdb/db/engine/memtable/IMemTable.java @@ -31,7 +31,6 @@ import java.io.IOException; import java.util.List; import java.util.Map; -import java.util.Set; /** * IMemTable is designed to store data points which are not flushed into TsFile yet. An instance of @@ -141,8 +140,4 @@ ReadOnlyMemChunk query( long getMaxPlanIndex(); long getMinPlanIndex(); - - - /** get devices from memTable * */ - Set getDevices(); } diff --git a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileProcessor.java b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileProcessor.java index 3d6501d13589a..12e7acc0bbb8b 100644 --- a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileProcessor.java +++ b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileProcessor.java @@ -1326,28 +1326,4 @@ public void submitAFlushTask() { public boolean alreadyMarkedClosing() { return shouldClose; } - - /** - * get all devices when the TsFile is not close. It consists of three part: 1. flushingMemTables, - * workMemtable is added into the flushingMemTables and memtables are ready to write to disk 2. - * workMemTable, the memtables ready to be written in memory 3. writer, the memtables which have - * been written to disk - * - * @return a set of all deviceId - */ - public Set getDevices() { - Set devicesSet = new HashSet<>(); - - for (IMemTable item : flushingMemTables) { - devicesSet.addAll(item.getDevices()); - } - if (workMemTable != null) { - devicesSet.addAll(workMemTable.getDevices()); - } - - if (writer != null) { - devicesSet.addAll(writer.getMetadatasForQuery().keySet()); - } - return devicesSet; - } } diff --git a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileResource.java b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileResource.java index e7689882d2543..42ea3a5de62e1 100644 --- a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileResource.java +++ b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileResource.java @@ -41,7 +41,6 @@ import org.apache.iotdb.tsfile.file.metadata.statistics.Statistics; import org.apache.iotdb.tsfile.fileSystem.FSFactoryProducer; import org.apache.iotdb.tsfile.fileSystem.fsFactory.FSFactory; -import org.apache.iotdb.tsfile.read.TsFileSequenceReader; import org.apache.iotdb.tsfile.read.filter.basic.Filter; import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils; @@ -464,7 +463,7 @@ public long getEndTime(String deviceId) { } public Set getDevices() { - return timeIndex.getDevices(file.getPath()); + return timeIndex.getDevices(file.getPath()); } public boolean endTimeEmpty() { diff --git a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/timeindex/FileTimeIndex.java b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/timeindex/FileTimeIndex.java index acc15d313d9db..fa23bad8d0761 100644 --- a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/timeindex/FileTimeIndex.java +++ b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/timeindex/FileTimeIndex.java @@ -122,8 +122,7 @@ public void close() { @Override public Set getDevices(String tsFilePath) { try { - TsFileSequenceReader fileReader = - FileReaderManager.getInstance().get(tsFilePath, true); + TsFileSequenceReader fileReader = FileReaderManager.getInstance().get(tsFilePath, true); return new HashSet<>(fileReader.getAllDevices()); } catch (IOException e) { logger.error("Can't read file {} from disk ", tsFilePath, e); diff --git a/server/src/test/java/org/apache/iotdb/db/engine/merge/MaxFileMergeFileSelectorTest.java b/server/src/test/java/org/apache/iotdb/db/engine/merge/MaxFileMergeFileSelectorTest.java index 4f5dadbe057ca..a91ae50d4f3b0 100644 --- a/server/src/test/java/org/apache/iotdb/db/engine/merge/MaxFileMergeFileSelectorTest.java +++ b/server/src/test/java/org/apache/iotdb/db/engine/merge/MaxFileMergeFileSelectorTest.java @@ -26,7 +26,6 @@ import org.apache.iotdb.db.engine.merge.selector.IMergeFileSelector; import org.apache.iotdb.db.engine.merge.selector.MaxFileMergeFileSelector; import org.apache.iotdb.db.engine.storagegroup.TsFileResource; -import org.apache.iotdb.db.engine.storagegroup.timeindex.DeviceTimeIndex; import org.apache.iotdb.db.engine.storagegroup.timeindex.ITimeIndex; import org.apache.iotdb.db.exception.MergeException; import org.apache.iotdb.tsfile.exception.write.WriteProcessException; @@ -132,7 +131,8 @@ public void testFileOpenSelection() Field timeIndexField = TsFileResource.class.getDeclaredField("timeIndex"); timeIndexField.setAccessible(true); ITimeIndex timeIndex = (ITimeIndex) timeIndexField.get(secondTsFileResource); - ITimeIndex newTimeIndex = IoTDBDescriptor.getInstance().getConfig().getTimeIndexLevel().getTimeIndex(); + ITimeIndex newTimeIndex = + IoTDBDescriptor.getInstance().getConfig().getTimeIndexLevel().getTimeIndex(); for (String device : devices) { newTimeIndex.updateStartTime(device, timeIndex.getStartTime(device)); @@ -184,7 +184,8 @@ public void testFileOpenSelectionFromCompaction() Field timeIndexField = TsFileResource.class.getDeclaredField("timeIndex"); timeIndexField.setAccessible(true); ITimeIndex timeIndex = (ITimeIndex) timeIndexField.get(secondTsFileResource); - ITimeIndex newTimeIndex = IoTDBDescriptor.getInstance().getConfig().getTimeIndexLevel().getTimeIndex(); + ITimeIndex newTimeIndex = + IoTDBDescriptor.getInstance().getConfig().getTimeIndexLevel().getTimeIndex(); for (String device : devices) { newTimeIndex.updateStartTime(device, timeIndex.getStartTime(device)); diff --git a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBLoadExternalTsfileIT.java b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBLoadExternalTsfileIT.java index 7caffe4f7c198..93e44d21a4bab 100644 --- a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBLoadExternalTsfileIT.java +++ b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBLoadExternalTsfileIT.java @@ -460,30 +460,30 @@ public void loadUnsequenceTsfileTest() throws SQLException { if (config.getTimeIndexLevel().equals(TimeIndexLevel.DEVICE_TIME_INDEX)) { assertEquals( - 1, - StorageEngine.getInstance() - .getProcessor(new PartialPath("root.test")) - .getUnSequenceFileList() - .size()); + 1, + StorageEngine.getInstance() + .getProcessor(new PartialPath("root.test")) + .getUnSequenceFileList() + .size()); assertEquals( - 3, - StorageEngine.getInstance() - .getProcessor(new PartialPath("root.test")) - .getSequenceFileTreeSet() - .size()); + 3, + StorageEngine.getInstance() + .getProcessor(new PartialPath("root.test")) + .getSequenceFileTreeSet() + .size()); } else if (config.getTimeIndexLevel().equals(TimeIndexLevel.FILE_TIME_INDEX)) { assertEquals( - 2, - StorageEngine.getInstance() - .getProcessor(new PartialPath("root.test")) - .getUnSequenceFileList() - .size()); + 2, + StorageEngine.getInstance() + .getProcessor(new PartialPath("root.test")) + .getUnSequenceFileList() + .size()); assertEquals( - 2, - StorageEngine.getInstance() - .getProcessor(new PartialPath("root.test")) - .getSequenceFileTreeSet() - .size()); + 2, + StorageEngine.getInstance() + .getProcessor(new PartialPath("root.test")) + .getSequenceFileTreeSet() + .size()); } assertNotNull(tmpDir.listFiles()); From 847b947c17a0271375593da748b22f0b46d13d2f Mon Sep 17 00:00:00 2001 From: Yuting Date: Tue, 27 Jul 2021 18:22:11 -0500 Subject: [PATCH 20/28] remove devices field in FileTimeIndex class --- .../org/apache/iotdb/db/conf/IoTDBConfig.java | 1 + .../db/engine/memtable/AbstractMemTable.java | 6 +- .../engine/storagegroup/TsFileProcessor.java | 6 +- .../engine/storagegroup/TsFileResource.java | 14 +++-- .../storagegroup/timeindex/FileTimeIndex.java | 58 ++----------------- .../merge/MaxFileMergeFileSelectorTest.java | 2 - .../IoTDBLoadExternalTsfileIT.java | 2 - 7 files changed, 26 insertions(+), 63 deletions(-) diff --git a/server/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java b/server/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java index bf395a96c5fef..4b8d6b869bea9 100644 --- a/server/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java +++ b/server/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java @@ -639,6 +639,7 @@ public class IoTDBConfig { * DEVICE_TIME_INDEX and FILE_TIME_INDEX are supported, and could not be changed after first set. */ private TimeIndexLevel timeIndexLevel = TimeIndexLevel.DEVICE_TIME_INDEX; + // just for test // wait for 60 second by default. private int thriftServerAwaitTimeForStopService = 60; diff --git a/server/src/main/java/org/apache/iotdb/db/engine/memtable/AbstractMemTable.java b/server/src/main/java/org/apache/iotdb/db/engine/memtable/AbstractMemTable.java index 32683c6f69333..0acb9e11e7f08 100644 --- a/server/src/main/java/org/apache/iotdb/db/engine/memtable/AbstractMemTable.java +++ b/server/src/main/java/org/apache/iotdb/db/engine/memtable/AbstractMemTable.java @@ -37,7 +37,11 @@ import org.apache.iotdb.tsfile.write.schema.VectorMeasurementSchema; import java.io.IOException; -import java.util.*; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; import java.util.Map.Entry; public abstract class AbstractMemTable implements IMemTable { diff --git a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileProcessor.java b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileProcessor.java index 12e7acc0bbb8b..ff19c8991b1ab 100644 --- a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileProcessor.java +++ b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileProcessor.java @@ -72,7 +72,11 @@ import java.io.File; import java.io.IOException; -import java.util.*; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Iterator; +import java.util.List; +import java.util.Set; import java.util.concurrent.ConcurrentLinkedDeque; import java.util.concurrent.locks.ReadWriteLock; import java.util.concurrent.locks.ReentrantReadWriteLock; diff --git a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileResource.java b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileResource.java index 42ea3a5de62e1..1ca377a900e9d 100644 --- a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileResource.java +++ b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileResource.java @@ -54,7 +54,13 @@ import java.nio.file.FileAlreadyExistsException; import java.nio.file.Files; import java.nio.file.Paths; -import java.util.*; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Random; +import java.util.Set; import static org.apache.iotdb.db.conf.IoTDBConstant.FILE_NAME_SEPARATOR; import static org.apache.iotdb.db.conf.IoTDBConstant.FILE_NAME_SUFFIX_INDEX; @@ -169,7 +175,7 @@ public TsFileResource(TsFileResource other) throws IOException { this.version = FilePathUtils.splitAndGetTsFileVersion(this.file.getName()); } - /** for sealed TsFile, call setClosed to close TsFileResource */ + /** for sealed TsFile, call setClosed to close TsFileResource * */ public TsFileResource(File file) { this.file = file; this.version = FilePathUtils.splitAndGetTsFileVersion(this.file.getName()); @@ -177,7 +183,7 @@ public TsFileResource(File file) { this.timeIndexType = (byte) config.getTimeIndexLevel().ordinal(); } - /** unsealed TsFile */ + /** unsealed TsFile, for writter */ public TsFileResource(File file, TsFileProcessor processor) { this.file = file; this.version = FilePathUtils.splitAndGetTsFileVersion(this.file.getName()); @@ -186,7 +192,7 @@ public TsFileResource(File file, TsFileProcessor processor) { this.processor = processor; } - /** unsealed TsFile */ + /** unsealed TsFile, for query */ public TsFileResource( List readOnlyMemChunk, List chunkMetadataList, diff --git a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/timeindex/FileTimeIndex.java b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/timeindex/FileTimeIndex.java index fa23bad8d0761..4eed523a6f39c 100644 --- a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/timeindex/FileTimeIndex.java +++ b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/timeindex/FileTimeIndex.java @@ -22,14 +22,11 @@ import org.apache.iotdb.db.engine.StorageEngine; import org.apache.iotdb.db.exception.PartitionViolationException; import org.apache.iotdb.db.query.control.FileReaderManager; -import org.apache.iotdb.db.rescon.CachedStringPool; import org.apache.iotdb.db.utils.FilePathUtils; -import org.apache.iotdb.db.utils.SerializeUtils; import org.apache.iotdb.tsfile.read.TsFileSequenceReader; import org.apache.iotdb.tsfile.utils.RamUsageEstimator; import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils; -import io.netty.util.internal.ConcurrentSet; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -39,14 +36,11 @@ import java.nio.ByteBuffer; import java.util.Collections; import java.util.HashSet; -import java.util.Map; import java.util.Set; public class FileTimeIndex implements ITimeIndex { private static final Logger logger = LoggerFactory.getLogger(FileTimeIndex.class); - protected static final Map cachedDevicePool = - CachedStringPool.getInstance().getCachedPool(); /** start time */ protected long startTime; @@ -54,64 +48,31 @@ public class FileTimeIndex implements ITimeIndex { /** end times. The value is Long.MIN_VALUE if it's an unsealed sequence tsfile */ protected long endTime; - /** devices */ - protected Set devices; - public FileTimeIndex() { - this.devices = new ConcurrentSet<>(); this.startTime = Long.MAX_VALUE; this.endTime = Long.MIN_VALUE; } - public FileTimeIndex(Set devices, long startTime, long endTime) { + public FileTimeIndex(long startTime, long endTime) { this.startTime = startTime; this.endTime = endTime; - this.devices = devices; } @Override public void serialize(OutputStream outputStream) throws IOException { - ReadWriteIOUtils.write(devices.size(), outputStream); - Set stringMemoryReducedSet = new ConcurrentSet<>(); - for (String device : devices) { - // To reduce the String number in memory, - // use the deviceId from cached pool - stringMemoryReducedSet.add(cachedDevicePool.computeIfAbsent(device, k -> k)); - ReadWriteIOUtils.write(device, outputStream); - } ReadWriteIOUtils.write(startTime, outputStream); ReadWriteIOUtils.write(endTime, outputStream); - devices = stringMemoryReducedSet; } @Override public FileTimeIndex deserialize(InputStream inputStream) throws IOException { - int size = ReadWriteIOUtils.readInt(inputStream); - Set deviceSet = new HashSet<>(); - for (int i = 0; i < size; i++) { - String path = ReadWriteIOUtils.readString(inputStream); - // To reduce the String number in memory, - // use the deviceId from memory instead of the deviceId read from disk - String cachedPath = cachedDevicePool.computeIfAbsent(path, k -> k); - deviceSet.add(cachedPath); - } return new FileTimeIndex( - deviceSet, ReadWriteIOUtils.readLong(inputStream), ReadWriteIOUtils.readLong(inputStream)); + ReadWriteIOUtils.readLong(inputStream), ReadWriteIOUtils.readLong(inputStream)); } @Override public FileTimeIndex deserialize(ByteBuffer buffer) { - int size = buffer.getInt(); - Set deviceSet = new HashSet<>(size); - - for (int i = 0; i < size; i++) { - String path = SerializeUtils.deserializeString(buffer); - // To reduce the String number in memory, - // use the deviceId from memory instead of the deviceId read from disk - String cachedPath = cachedDevicePool.computeIfAbsent(path, k -> k); - deviceSet.add(cachedPath); - } - return new FileTimeIndex(deviceSet, buffer.getLong(), buffer.getLong()); + return new FileTimeIndex(buffer.getLong(), buffer.getLong()); } @Override @@ -128,7 +89,6 @@ public Set getDevices(String tsFilePath) { logger.error("Can't read file {} from disk ", tsFilePath, e); } return Collections.emptySet(); - // return devices; } @Override @@ -147,17 +107,12 @@ public boolean stillLives(long ttlLowerBound) { @Override public long calculateRamSize() { - return RamUsageEstimator.sizeOf(devices) - + RamUsageEstimator.sizeOf(startTime) - + RamUsageEstimator.sizeOf(endTime); + return RamUsageEstimator.sizeOf(startTime) + RamUsageEstimator.sizeOf(endTime); } @Override public long getTimePartition(String tsFilePath) { try { - if (devices != null && !devices.isEmpty()) { - return StorageEngine.getTimePartition(startTime); - } String[] filePathSplits = FilePathUtils.splitTsFilePath(tsFilePath); return Long.parseLong(filePathSplits[filePathSplits.length - 2]); } catch (NumberFormatException e) { @@ -191,7 +146,7 @@ public boolean isSpanMultiTimePartitions() { @Override public void updateStartTime(String deviceId, long time) { - devices.add(deviceId); + if (this.startTime > time) { this.startTime = time; } @@ -199,7 +154,6 @@ public void updateStartTime(String deviceId, long time) { @Override public void updateEndTime(String deviceId, long time) { - devices.add(deviceId); if (this.endTime < time) { this.endTime = time; } @@ -207,13 +161,11 @@ public void updateEndTime(String deviceId, long time) { @Override public void putStartTime(String deviceId, long time) { - devices.add(deviceId); this.startTime = time; } @Override public void putEndTime(String deviceId, long time) { - devices.add(deviceId); this.endTime = time; } diff --git a/server/src/test/java/org/apache/iotdb/db/engine/merge/MaxFileMergeFileSelectorTest.java b/server/src/test/java/org/apache/iotdb/db/engine/merge/MaxFileMergeFileSelectorTest.java index a91ae50d4f3b0..f852a88122309 100644 --- a/server/src/test/java/org/apache/iotdb/db/engine/merge/MaxFileMergeFileSelectorTest.java +++ b/server/src/test/java/org/apache/iotdb/db/engine/merge/MaxFileMergeFileSelectorTest.java @@ -133,7 +133,6 @@ public void testFileOpenSelection() ITimeIndex timeIndex = (ITimeIndex) timeIndexField.get(secondTsFileResource); ITimeIndex newTimeIndex = IoTDBDescriptor.getInstance().getConfig().getTimeIndexLevel().getTimeIndex(); - for (String device : devices) { newTimeIndex.updateStartTime(device, timeIndex.getStartTime(device)); } @@ -186,7 +185,6 @@ public void testFileOpenSelectionFromCompaction() ITimeIndex timeIndex = (ITimeIndex) timeIndexField.get(secondTsFileResource); ITimeIndex newTimeIndex = IoTDBDescriptor.getInstance().getConfig().getTimeIndexLevel().getTimeIndex(); - for (String device : devices) { newTimeIndex.updateStartTime(device, timeIndex.getStartTime(device)); } diff --git a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBLoadExternalTsfileIT.java b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBLoadExternalTsfileIT.java index 93e44d21a4bab..f1a149f155600 100644 --- a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBLoadExternalTsfileIT.java +++ b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBLoadExternalTsfileIT.java @@ -457,7 +457,6 @@ public void loadUnsequenceTsfileTest() throws SQLException { .getProcessor(new PartialPath("root.vehicle")) .getUnSequenceFileList() .size()); - if (config.getTimeIndexLevel().equals(TimeIndexLevel.DEVICE_TIME_INDEX)) { assertEquals( 1, @@ -485,7 +484,6 @@ public void loadUnsequenceTsfileTest() throws SQLException { .getSequenceFileTreeSet() .size()); } - assertNotNull(tmpDir.listFiles()); assertEquals( 0, From 6b52437fc858fe209cfd46c0009221f61fa1f1a1 Mon Sep 17 00:00:00 2001 From: Yuting Date: Tue, 27 Jul 2021 19:53:21 -0500 Subject: [PATCH 21/28] modify the comments --- .../org/apache/iotdb/db/engine/storagegroup/TsFileResource.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileResource.java b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileResource.java index 1ca377a900e9d..b6f07d9e1756c 100644 --- a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileResource.java +++ b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileResource.java @@ -175,7 +175,7 @@ public TsFileResource(TsFileResource other) throws IOException { this.version = FilePathUtils.splitAndGetTsFileVersion(this.file.getName()); } - /** for sealed TsFile, call setClosed to close TsFileResource * */ + /** for sealed TsFile, call setClosed to close TsFileResource */ public TsFileResource(File file) { this.file = file; this.version = FilePathUtils.splitAndGetTsFileVersion(this.file.getName()); From c928c187ba43fc46ec552b86f8073fbca869ec3c Mon Sep 17 00:00:00 2001 From: Yuting Date: Thu, 5 Aug 2021 14:35:38 -0500 Subject: [PATCH 22/28] add throw RunTimeException --- example/mqtt-customize/pom.xml | 1 - 1 file changed, 1 deletion(-) diff --git a/example/mqtt-customize/pom.xml b/example/mqtt-customize/pom.xml index 7098667fc840b..c524fb0ad853c 100644 --- a/example/mqtt-customize/pom.xml +++ b/example/mqtt-customize/pom.xml @@ -37,6 +37,5 @@ iotdb-server ${project.version} - From c9227cd5047a37cfc49f815269469a9a3a908cfa Mon Sep 17 00:00:00 2001 From: Yuting Date: Thu, 5 Aug 2021 14:40:08 -0500 Subject: [PATCH 23/28] add throw RunTimeException --- .../iotdb/db/engine/storagegroup/timeindex/FileTimeIndex.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/timeindex/FileTimeIndex.java b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/timeindex/FileTimeIndex.java index 4eed523a6f39c..d2e5b438b53e7 100644 --- a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/timeindex/FileTimeIndex.java +++ b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/timeindex/FileTimeIndex.java @@ -87,8 +87,8 @@ public Set getDevices(String tsFilePath) { return new HashSet<>(fileReader.getAllDevices()); } catch (IOException e) { logger.error("Can't read file {} from disk ", tsFilePath, e); + throw new RuntimeException("Can't read file" + tsFilePath + "from disk" ); } - return Collections.emptySet(); } @Override From 032601c173b5e5463ee44ade83e77746e29fbfa7 Mon Sep 17 00:00:00 2001 From: Yuting Date: Thu, 5 Aug 2021 20:31:21 -0500 Subject: [PATCH 24/28] spotless:apply --- .../iotdb/db/engine/storagegroup/timeindex/FileTimeIndex.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/timeindex/FileTimeIndex.java b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/timeindex/FileTimeIndex.java index d2e5b438b53e7..fe4ab64eea44a 100644 --- a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/timeindex/FileTimeIndex.java +++ b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/timeindex/FileTimeIndex.java @@ -34,7 +34,6 @@ import java.io.InputStream; import java.io.OutputStream; import java.nio.ByteBuffer; -import java.util.Collections; import java.util.HashSet; import java.util.Set; @@ -87,7 +86,7 @@ public Set getDevices(String tsFilePath) { return new HashSet<>(fileReader.getAllDevices()); } catch (IOException e) { logger.error("Can't read file {} from disk ", tsFilePath, e); - throw new RuntimeException("Can't read file" + tsFilePath + "from disk" ); + throw new RuntimeException("Can't read file" + tsFilePath + "from disk"); } } From c37cfe519f12fd1e6d60aab93803292e7f967d37 Mon Sep 17 00:00:00 2001 From: Yuting Date: Thu, 5 Aug 2021 21:38:21 -0500 Subject: [PATCH 25/28] add throwing IOException for getDevices of FileTimeIndex --- .../cluster/log/snapshot/FileSnapshot.java | 4 +-- .../apache/iotdb/db/engine/StorageEngine.java | 4 +-- .../storagegroup/StorageGroupProcessor.java | 28 +++++++++++-------- .../engine/storagegroup/TsFileResource.java | 2 +- .../storagegroup/timeindex/FileTimeIndex.java | 7 +---- .../storagegroup/timeindex/ITimeIndex.java | 2 +- 6 files changed, 24 insertions(+), 23 deletions(-) diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/log/snapshot/FileSnapshot.java b/cluster/src/main/java/org/apache/iotdb/cluster/log/snapshot/FileSnapshot.java index a67cd50e5996d..2d4f7bc443365 100644 --- a/cluster/src/main/java/org/apache/iotdb/cluster/log/snapshot/FileSnapshot.java +++ b/cluster/src/main/java/org/apache/iotdb/cluster/log/snapshot/FileSnapshot.java @@ -294,7 +294,7 @@ private void installFileSnapshotFiles(FileSnapshot snapshot, int slot, boolean i removeRemoteHardLink(resource); } } - } catch (IllegalPathException e) { + } catch (IllegalPathException | IOException e) { throw new PullFileException(resource.getTsFilePath(), resource.getSource(), e); } } @@ -313,7 +313,7 @@ private void installFileSnapshotFiles(FileSnapshot snapshot, int slot, boolean i * @param resource * @return */ - private boolean isFileAlreadyPulled(RemoteTsFileResource resource) throws IllegalPathException { + private boolean isFileAlreadyPulled(RemoteTsFileResource resource) throws IllegalPathException, IOException { Pair sgNameAndTimePartitionIdPair = FilePathUtils.getLogicalSgNameAndTimePartitionIdPair(resource); return StorageEngine.getInstance() diff --git a/server/src/main/java/org/apache/iotdb/db/engine/StorageEngine.java b/server/src/main/java/org/apache/iotdb/db/engine/StorageEngine.java index ef955f5232357..90da753460aaa 100644 --- a/server/src/main/java/org/apache/iotdb/db/engine/StorageEngine.java +++ b/server/src/main/java/org/apache/iotdb/db/engine/StorageEngine.java @@ -789,7 +789,7 @@ public void loadNewTsFileForSync(TsFileResource newTsFileResource) } public void loadNewTsFile(TsFileResource newTsFileResource) - throws LoadFileException, StorageEngineException, MetadataException { + throws LoadFileException, StorageEngineException, MetadataException, IOException { Set deviceSet = newTsFileResource.getDevices(); if (deviceSet == null || deviceSet.isEmpty()) { throw new StorageEngineException("Can not get the corresponding storage group."); @@ -843,7 +843,7 @@ public void setFileFlushPolicy(TsFileFlushPolicy fileFlushPolicy) { } public boolean isFileAlreadyExist( - TsFileResource tsFileResource, PartialPath storageGroup, long partitionNum) { + TsFileResource tsFileResource, PartialPath storageGroup, long partitionNum) throws IOException { VirtualStorageGroupManager virtualStorageGroupManager = processorMap.get(storageGroup); if (virtualStorageGroupManager == null) { return false; diff --git a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/StorageGroupProcessor.java b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/StorageGroupProcessor.java index af54491064735..51d9182ee6ba8 100755 --- a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/StorageGroupProcessor.java +++ b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/StorageGroupProcessor.java @@ -387,7 +387,7 @@ public StorageGroupProcessor( String virtualStorageGroupId, TsFileFlushPolicy fileFlushPolicy, String logicalStorageGroupName) - throws StorageGroupProcessorException { + throws StorageGroupProcessorException, IOException { this.virtualStorageGroupId = virtualStorageGroupId; this.logicalStorageGroupName = logicalStorageGroupName; this.fileFlushPolicy = fileFlushPolicy; @@ -437,7 +437,7 @@ private Map> splitResourcesByPartition( } /** recover from file */ - private void recover() throws StorageGroupProcessorException { + private void recover() throws StorageGroupProcessorException, IOException { logger.info( String.format( "start recovering virtual storage group %s[%s]", @@ -1332,7 +1332,7 @@ public void syncCloseOneTsFileProcessor(boolean sequence, TsFileProcessor tsFile (System.currentTimeMillis() - startTime) / 1000); } } - } catch (InterruptedException e) { + } catch (InterruptedException | IOException e) { Thread.currentThread().interrupt(); logger.error( "syncCloseOneTsFileProcessor error occurs while waiting for closing the storage " @@ -1349,7 +1349,7 @@ public void syncCloseOneTsFileProcessor(boolean sequence, TsFileProcessor tsFile * @param sequence whether this tsfile processor is sequence or not * @param tsFileProcessor tsfile processor */ - public void asyncCloseOneTsFileProcessor(boolean sequence, TsFileProcessor tsFileProcessor) { + public void asyncCloseOneTsFileProcessor(boolean sequence, TsFileProcessor tsFileProcessor) throws IOException { // 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 (closingSequenceTsFileProcessor.contains(tsFileProcessor) @@ -1618,6 +1618,8 @@ public void asyncCloseAllWorkingTsFileProcessors() { new ArrayList<>(workUnsequenceTsFileProcessors.values())) { asyncCloseOneTsFileProcessor(false, tsFileProcessor); } + } catch (IOException e) { + logger.error("Can't read file {} from disk." ); } finally { writeUnlock(); } @@ -1959,7 +1961,7 @@ private void tryToDeleteLastCache( * * @param tsFileProcessor processor to be closed */ - private void updateEndTimeMap(TsFileProcessor tsFileProcessor) { + private void updateEndTimeMap(TsFileProcessor tsFileProcessor) throws IOException { TsFileResource resource = tsFileProcessor.getTsFileResource(); for (String deviceId : resource.getDevices()) { resource.updateEndTime( @@ -2135,7 +2137,7 @@ public void upgrade() { } } - private void upgradeTsFileResourceCallBack(TsFileResource tsFileResource) { + private void upgradeTsFileResourceCallBack(TsFileResource tsFileResource) throws IOException { List upgradedResources = tsFileResource.getUpgradedResources(); for (TsFileResource resource : upgradedResources) { long partitionId = resource.getTimePartition(); @@ -2261,13 +2263,15 @@ public void loadNewTsFileForSync(TsFileResource newTsFileResource) throws LoadFi logger.error( "Failed to reset last cache when loading file {}", newTsFileResource.getTsFilePath()); throw new LoadFileException(e); + } catch (IOException e) { + e.printStackTrace(); } finally { writeUnlock(); } } private void resetLastCacheWhenLoadingTsfile(TsFileResource newTsFileResource) - throws IllegalPathException { + throws IllegalPathException, IOException { for (String device : newTsFileResource.getDevices()) { tryToDeleteLastCacheByDevice(new PartialPath(device)); } @@ -2363,6 +2367,8 @@ public void loadNewTsFile(TsFileResource newTsFileResource) throws LoadFileExcep logger.error( "Failed to reset last cache when loading file {}", newTsFileResource.getTsFilePath()); throw new LoadFileException(e); + } catch (IOException e) { + e.printStackTrace(); } finally { writeUnlock(); } @@ -2393,7 +2399,7 @@ private long computeMaxVersion(Long oldVersion, Long newVersion) { private int findInsertionPosition( TsFileResource newTsFileResource, long newFilePartitionId, - List sequenceList) { + List sequenceList) throws IOException { int insertPos = -1; @@ -2434,7 +2440,7 @@ private int findInsertionPosition( * @return -1 if fileA is totally older than fileB (A < B) 0 if fileA is partially older than * fileB and partially newer than fileB (A X B) 1 if fileA is totally newer than fileB (B < A) */ - private int compareTsFileDevices(TsFileResource fileA, TsFileResource fileB) { + private int compareTsFileDevices(TsFileResource fileA, TsFileResource fileB) throws IOException { boolean hasPre = false, hasSubsequence = false; for (String device : fileA.getDevices()) { if (!fileB.getDevices().contains(device)) { @@ -2606,7 +2612,7 @@ private long getAndSetNewVersion(long timePartitionId, TsFileResource tsFileReso * Update latest time in latestTimeForEachDevice and * partitionLatestFlushedTimeForEachDevice. @UsedBy sync module, load external tsfile module. */ - private void updateLatestTimeMap(TsFileResource newTsFileResource) { + private void updateLatestTimeMap(TsFileResource newTsFileResource) throws IOException { for (String device : newTsFileResource.getDevices()) { long endTime = newTsFileResource.getEndTime(device); long timePartitionId = StorageEngine.getTimePartition(endTime); @@ -3118,7 +3124,7 @@ public interface UpdateEndTimeCallBack { @FunctionalInterface public interface UpgradeTsFileResourceCallBack { - void call(TsFileResource caller); + void call(TsFileResource caller) throws IOException; } @FunctionalInterface diff --git a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileResource.java b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileResource.java index b6f07d9e1756c..334b2d8520c7c 100644 --- a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileResource.java +++ b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileResource.java @@ -468,7 +468,7 @@ public long getEndTime(String deviceId) { return timeIndex.getEndTime(deviceId); } - public Set getDevices() { + public Set getDevices() throws IOException { return timeIndex.getDevices(file.getPath()); } diff --git a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/timeindex/FileTimeIndex.java b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/timeindex/FileTimeIndex.java index d2e5b438b53e7..18c6015dcb8d5 100644 --- a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/timeindex/FileTimeIndex.java +++ b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/timeindex/FileTimeIndex.java @@ -81,14 +81,9 @@ public void close() { } @Override - public Set getDevices(String tsFilePath) { - try { + public Set getDevices(String tsFilePath) throws IOException { TsFileSequenceReader fileReader = FileReaderManager.getInstance().get(tsFilePath, true); return new HashSet<>(fileReader.getAllDevices()); - } catch (IOException e) { - logger.error("Can't read file {} from disk ", tsFilePath, e); - throw new RuntimeException("Can't read file" + tsFilePath + "from disk" ); - } } @Override diff --git a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/timeindex/ITimeIndex.java b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/timeindex/ITimeIndex.java index d53dd48515152..654f02ec06a7f 100644 --- a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/timeindex/ITimeIndex.java +++ b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/timeindex/ITimeIndex.java @@ -62,7 +62,7 @@ public interface ITimeIndex { * * @return device names */ - Set getDevices(String tsFilePath); + Set getDevices(String tsFilePath) throws IOException; /** @return whether end time is empty (Long.MIN_VALUE) */ boolean endTimeEmpty(); From 15070832475cae8b70913fe01561b471bfac5b21 Mon Sep 17 00:00:00 2001 From: Yuting Date: Fri, 6 Aug 2021 10:02:34 -0500 Subject: [PATCH 26/28] format change --- .../iotdb/db/engine/storagegroup/timeindex/FileTimeIndex.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/timeindex/FileTimeIndex.java b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/timeindex/FileTimeIndex.java index fe4ab64eea44a..d427f34a0b699 100644 --- a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/timeindex/FileTimeIndex.java +++ b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/timeindex/FileTimeIndex.java @@ -86,7 +86,7 @@ public Set getDevices(String tsFilePath) { return new HashSet<>(fileReader.getAllDevices()); } catch (IOException e) { logger.error("Can't read file {} from disk ", tsFilePath, e); - throw new RuntimeException("Can't read file" + tsFilePath + "from disk"); + throw new RuntimeException("Can't read file " + tsFilePath + " from disk"); } } @@ -145,7 +145,6 @@ public boolean isSpanMultiTimePartitions() { @Override public void updateStartTime(String deviceId, long time) { - if (this.startTime > time) { this.startTime = time; } From 59c9a76cf4d113a15a5476a484779a46a8cbd4ea Mon Sep 17 00:00:00 2001 From: Yuting Date: Sun, 8 Aug 2021 21:55:04 -0500 Subject: [PATCH 27/28] delete part of IOException throw --- .../cluster/log/snapshot/FileSnapshot.java | 4 +-- .../apache/iotdb/db/engine/StorageEngine.java | 2 +- .../storagegroup/StorageGroupProcessor.java | 28 ++++++++----------- 3 files changed, 14 insertions(+), 20 deletions(-) diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/log/snapshot/FileSnapshot.java b/cluster/src/main/java/org/apache/iotdb/cluster/log/snapshot/FileSnapshot.java index 2d4f7bc443365..b4ccc0785276e 100644 --- a/cluster/src/main/java/org/apache/iotdb/cluster/log/snapshot/FileSnapshot.java +++ b/cluster/src/main/java/org/apache/iotdb/cluster/log/snapshot/FileSnapshot.java @@ -294,7 +294,7 @@ private void installFileSnapshotFiles(FileSnapshot snapshot, int slot, boolean i removeRemoteHardLink(resource); } } - } catch (IllegalPathException | IOException e) { + } catch (IllegalPathException) { throw new PullFileException(resource.getTsFilePath(), resource.getSource(), e); } } @@ -313,7 +313,7 @@ private void installFileSnapshotFiles(FileSnapshot snapshot, int slot, boolean i * @param resource * @return */ - private boolean isFileAlreadyPulled(RemoteTsFileResource resource) throws IllegalPathException, IOException { + private boolean isFileAlreadyPulled(RemoteTsFileResource resource) throws IllegalPathException { Pair sgNameAndTimePartitionIdPair = FilePathUtils.getLogicalSgNameAndTimePartitionIdPair(resource); return StorageEngine.getInstance() diff --git a/server/src/main/java/org/apache/iotdb/db/engine/StorageEngine.java b/server/src/main/java/org/apache/iotdb/db/engine/StorageEngine.java index 90da753460aaa..d38478f2a7702 100644 --- a/server/src/main/java/org/apache/iotdb/db/engine/StorageEngine.java +++ b/server/src/main/java/org/apache/iotdb/db/engine/StorageEngine.java @@ -843,7 +843,7 @@ public void setFileFlushPolicy(TsFileFlushPolicy fileFlushPolicy) { } public boolean isFileAlreadyExist( - TsFileResource tsFileResource, PartialPath storageGroup, long partitionNum) throws IOException { + TsFileResource tsFileResource, PartialPath storageGroup, long partitionNum) { VirtualStorageGroupManager virtualStorageGroupManager = processorMap.get(storageGroup); if (virtualStorageGroupManager == null) { return false; diff --git a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/StorageGroupProcessor.java b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/StorageGroupProcessor.java index 51d9182ee6ba8..be3699d5da7b2 100755 --- a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/StorageGroupProcessor.java +++ b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/StorageGroupProcessor.java @@ -387,7 +387,7 @@ public StorageGroupProcessor( String virtualStorageGroupId, TsFileFlushPolicy fileFlushPolicy, String logicalStorageGroupName) - throws StorageGroupProcessorException, IOException { + throws StorageGroupProcessorException{ this.virtualStorageGroupId = virtualStorageGroupId; this.logicalStorageGroupName = logicalStorageGroupName; this.fileFlushPolicy = fileFlushPolicy; @@ -437,7 +437,7 @@ private Map> splitResourcesByPartition( } /** recover from file */ - private void recover() throws StorageGroupProcessorException, IOException { + private void recover() throws StorageGroupProcessorException { logger.info( String.format( "start recovering virtual storage group %s[%s]", @@ -1332,7 +1332,7 @@ public void syncCloseOneTsFileProcessor(boolean sequence, TsFileProcessor tsFile (System.currentTimeMillis() - startTime) / 1000); } } - } catch (InterruptedException | IOException e) { + } catch (InterruptedException e) { Thread.currentThread().interrupt(); logger.error( "syncCloseOneTsFileProcessor error occurs while waiting for closing the storage " @@ -1349,7 +1349,7 @@ public void syncCloseOneTsFileProcessor(boolean sequence, TsFileProcessor tsFile * @param sequence whether this tsfile processor is sequence or not * @param tsFileProcessor tsfile processor */ - public void asyncCloseOneTsFileProcessor(boolean sequence, TsFileProcessor tsFileProcessor) throws IOException { + public void asyncCloseOneTsFileProcessor(boolean sequence, TsFileProcessor tsFileProcessor) { // 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 (closingSequenceTsFileProcessor.contains(tsFileProcessor) @@ -1618,8 +1618,6 @@ public void asyncCloseAllWorkingTsFileProcessors() { new ArrayList<>(workUnsequenceTsFileProcessors.values())) { asyncCloseOneTsFileProcessor(false, tsFileProcessor); } - } catch (IOException e) { - logger.error("Can't read file {} from disk." ); } finally { writeUnlock(); } @@ -1961,7 +1959,7 @@ private void tryToDeleteLastCache( * * @param tsFileProcessor processor to be closed */ - private void updateEndTimeMap(TsFileProcessor tsFileProcessor) throws IOException { + private void updateEndTimeMap(TsFileProcessor tsFileProcessor) { TsFileResource resource = tsFileProcessor.getTsFileResource(); for (String deviceId : resource.getDevices()) { resource.updateEndTime( @@ -2137,7 +2135,7 @@ public void upgrade() { } } - private void upgradeTsFileResourceCallBack(TsFileResource tsFileResource) throws IOException { + private void upgradeTsFileResourceCallBack(TsFileResource tsFileResource) { List upgradedResources = tsFileResource.getUpgradedResources(); for (TsFileResource resource : upgradedResources) { long partitionId = resource.getTimePartition(); @@ -2263,15 +2261,13 @@ public void loadNewTsFileForSync(TsFileResource newTsFileResource) throws LoadFi logger.error( "Failed to reset last cache when loading file {}", newTsFileResource.getTsFilePath()); throw new LoadFileException(e); - } catch (IOException e) { - e.printStackTrace(); } finally { writeUnlock(); } } private void resetLastCacheWhenLoadingTsfile(TsFileResource newTsFileResource) - throws IllegalPathException, IOException { + throws IllegalPathException { for (String device : newTsFileResource.getDevices()) { tryToDeleteLastCacheByDevice(new PartialPath(device)); } @@ -2367,8 +2363,6 @@ public void loadNewTsFile(TsFileResource newTsFileResource) throws LoadFileExcep logger.error( "Failed to reset last cache when loading file {}", newTsFileResource.getTsFilePath()); throw new LoadFileException(e); - } catch (IOException e) { - e.printStackTrace(); } finally { writeUnlock(); } @@ -2399,7 +2393,7 @@ private long computeMaxVersion(Long oldVersion, Long newVersion) { private int findInsertionPosition( TsFileResource newTsFileResource, long newFilePartitionId, - List sequenceList) throws IOException { + List sequenceList) { int insertPos = -1; @@ -2440,7 +2434,7 @@ private int findInsertionPosition( * @return -1 if fileA is totally older than fileB (A < B) 0 if fileA is partially older than * fileB and partially newer than fileB (A X B) 1 if fileA is totally newer than fileB (B < A) */ - private int compareTsFileDevices(TsFileResource fileA, TsFileResource fileB) throws IOException { + private int compareTsFileDevices(TsFileResource fileA, TsFileResource fileB) { boolean hasPre = false, hasSubsequence = false; for (String device : fileA.getDevices()) { if (!fileB.getDevices().contains(device)) { @@ -2612,7 +2606,7 @@ private long getAndSetNewVersion(long timePartitionId, TsFileResource tsFileReso * Update latest time in latestTimeForEachDevice and * partitionLatestFlushedTimeForEachDevice. @UsedBy sync module, load external tsfile module. */ - private void updateLatestTimeMap(TsFileResource newTsFileResource) throws IOException { + private void updateLatestTimeMap(TsFileResource newTsFileResource) { for (String device : newTsFileResource.getDevices()) { long endTime = newTsFileResource.getEndTime(device); long timePartitionId = StorageEngine.getTimePartition(endTime); @@ -3124,7 +3118,7 @@ public interface UpdateEndTimeCallBack { @FunctionalInterface public interface UpgradeTsFileResourceCallBack { - void call(TsFileResource caller) throws IOException; + void call(TsFileResource caller); } @FunctionalInterface From aa88441f0d7fea160ae3e5a14fe6d5229b8afa2d Mon Sep 17 00:00:00 2001 From: Yuting Date: Sun, 8 Aug 2021 22:08:10 -0500 Subject: [PATCH 28/28] revise format --- .../org/apache/iotdb/cluster/log/snapshot/FileSnapshot.java | 2 +- .../main/java/org/apache/iotdb/db/engine/StorageEngine.java | 2 +- .../iotdb/db/engine/storagegroup/StorageGroupProcessor.java | 4 ++-- .../iotdb/db/engine/storagegroup/timeindex/FileTimeIndex.java | 4 ++-- 4 files changed, 6 insertions(+), 6 deletions(-) diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/log/snapshot/FileSnapshot.java b/cluster/src/main/java/org/apache/iotdb/cluster/log/snapshot/FileSnapshot.java index b4ccc0785276e..a67cd50e5996d 100644 --- a/cluster/src/main/java/org/apache/iotdb/cluster/log/snapshot/FileSnapshot.java +++ b/cluster/src/main/java/org/apache/iotdb/cluster/log/snapshot/FileSnapshot.java @@ -294,7 +294,7 @@ private void installFileSnapshotFiles(FileSnapshot snapshot, int slot, boolean i removeRemoteHardLink(resource); } } - } catch (IllegalPathException) { + } catch (IllegalPathException e) { throw new PullFileException(resource.getTsFilePath(), resource.getSource(), e); } } diff --git a/server/src/main/java/org/apache/iotdb/db/engine/StorageEngine.java b/server/src/main/java/org/apache/iotdb/db/engine/StorageEngine.java index d38478f2a7702..ef955f5232357 100644 --- a/server/src/main/java/org/apache/iotdb/db/engine/StorageEngine.java +++ b/server/src/main/java/org/apache/iotdb/db/engine/StorageEngine.java @@ -789,7 +789,7 @@ public void loadNewTsFileForSync(TsFileResource newTsFileResource) } public void loadNewTsFile(TsFileResource newTsFileResource) - throws LoadFileException, StorageEngineException, MetadataException, IOException { + throws LoadFileException, StorageEngineException, MetadataException { Set deviceSet = newTsFileResource.getDevices(); if (deviceSet == null || deviceSet.isEmpty()) { throw new StorageEngineException("Can not get the corresponding storage group."); diff --git a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/StorageGroupProcessor.java b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/StorageGroupProcessor.java index be3699d5da7b2..af54491064735 100755 --- a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/StorageGroupProcessor.java +++ b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/StorageGroupProcessor.java @@ -387,7 +387,7 @@ public StorageGroupProcessor( String virtualStorageGroupId, TsFileFlushPolicy fileFlushPolicy, String logicalStorageGroupName) - throws StorageGroupProcessorException{ + throws StorageGroupProcessorException { this.virtualStorageGroupId = virtualStorageGroupId; this.logicalStorageGroupName = logicalStorageGroupName; this.fileFlushPolicy = fileFlushPolicy; @@ -2267,7 +2267,7 @@ public void loadNewTsFileForSync(TsFileResource newTsFileResource) throws LoadFi } private void resetLastCacheWhenLoadingTsfile(TsFileResource newTsFileResource) - throws IllegalPathException { + throws IllegalPathException { for (String device : newTsFileResource.getDevices()) { tryToDeleteLastCacheByDevice(new PartialPath(device)); } diff --git a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/timeindex/FileTimeIndex.java b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/timeindex/FileTimeIndex.java index 94aeb49da910e..7f25deb12d289 100644 --- a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/timeindex/FileTimeIndex.java +++ b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/timeindex/FileTimeIndex.java @@ -81,8 +81,8 @@ public void close() { @Override public Set getDevices(String tsFilePath) throws IOException { - TsFileSequenceReader fileReader = FileReaderManager.getInstance().get(tsFilePath, true); - return new HashSet<>(fileReader.getAllDevices()); + TsFileSequenceReader fileReader = FileReaderManager.getInstance().get(tsFilePath, true); + return new HashSet<>(fileReader.getAllDevices()); } @Override