You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@iotdb.apache.org by hu...@apache.org on 2022/08/25 08:29:59 UTC
[iotdb] branch rel/0.13 updated: [To rel/0.13] [IOTDB-4196] Fix concurrent bug in TsFileSequenceReader (#7125)
This is an automated email from the ASF dual-hosted git repository.
hui pushed a commit to branch rel/0.13
in repository https://gitbox.apache.org/repos/asf/iotdb.git
The following commit(s) were added to refs/heads/rel/0.13 by this push:
new a281fa579a [To rel/0.13] [IOTDB-4196] Fix concurrent bug in TsFileSequenceReader (#7125)
a281fa579a is described below
commit a281fa579a9e05c9d6557fb91b7dbb00afc02889
Author: liuminghui233 <36...@users.noreply.github.com>
AuthorDate: Thu Aug 25 16:29:52 2022 +0800
[To rel/0.13] [IOTDB-4196] Fix concurrent bug in TsFileSequenceReader (#7125)
---
.../org/apache/iotdb/tsfile/read/TsFileSequenceReader.java | 10 +++++++---
1 file changed, 7 insertions(+), 3 deletions(-)
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/TsFileSequenceReader.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/TsFileSequenceReader.java
index 81d6c8a63b..619890a74c 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/TsFileSequenceReader.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/TsFileSequenceReader.java
@@ -100,7 +100,7 @@ public class TsFileSequenceReader implements AutoCloseable {
protected long fileMetadataPos;
protected int fileMetadataSize;
private ByteBuffer markerBuffer = ByteBuffer.allocate(Byte.BYTES);
- protected TsFileMetadata tsFileMetaData;
+ protected volatile TsFileMetadata tsFileMetaData;
// device -> measurement -> TimeseriesMetadata
private Map<String, Map<String, TimeseriesMetadata>> cachedDeviceMetadata =
new ConcurrentHashMap<>();
@@ -269,8 +269,12 @@ public class TsFileSequenceReader implements AutoCloseable {
public TsFileMetadata readFileMetadata() throws IOException {
try {
if (tsFileMetaData == null) {
- tsFileMetaData =
- TsFileMetadata.deserializeFrom(readData(fileMetadataPos, fileMetadataSize));
+ synchronized (this) {
+ if (tsFileMetaData == null) {
+ tsFileMetaData =
+ TsFileMetadata.deserializeFrom(readData(fileMetadataPos, fileMetadataSize));
+ }
+ }
}
} catch (Exception e) {
logger.error("Something error happened while reading file metadata of file {}", file);