You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@iotdb.apache.org by ja...@apache.org on 2021/01/08 06:15:12 UTC
[iotdb] 01/01: add error log to print file name while error happened
This is an automated email from the ASF dual-hosted git repository.
jackietien pushed a commit to branch MoreErrorLog
in repository https://gitbox.apache.org/repos/asf/iotdb.git
commit 6b59007b12141a44d166841f526054117e3e655d
Author: JackieTien97 <Ja...@foxmail.com>
AuthorDate: Fri Jan 8 14:14:32 2021 +0800
add error log to print file name while error happened
---
.../iotdb/tsfile/read/TsFileSequenceReader.java | 168 ++++++++++++++-------
1 file changed, 114 insertions(+), 54 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 6d79f9b..777893c 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
@@ -20,6 +20,7 @@ package org.apache.iotdb.tsfile.read;
import java.io.File;
import java.io.IOException;
+import java.nio.BufferOverflowException;
import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.Collections;
@@ -240,8 +241,14 @@ public class TsFileSequenceReader implements AutoCloseable {
* @throws IOException io error
*/
public TsFileMetadata readFileMetadata() throws IOException {
- if (tsFileMetaData == null) {
- tsFileMetaData = TsFileMetadata.deserializeFrom(readData(fileMetadataPos, fileMetadataSize));
+ try {
+ if (tsFileMetaData == null) {
+ tsFileMetaData = TsFileMetadata
+ .deserializeFrom(readData(fileMetadataPos, fileMetadataSize));
+ }
+ } catch (BufferOverflowException e) {
+ logger.error("Something error happened while reading file metadata of file {}", file, e);
+ throw e;
}
return tsFileMetaData;
}
@@ -313,7 +320,13 @@ public class TsFileSequenceReader implements AutoCloseable {
ByteBuffer buffer = readData(metadataIndexPair.left.getOffset(), metadataIndexPair.right);
MetadataIndexNode metadataIndexNode = deviceMetadataIndexNode;
if (!metadataIndexNode.getNodeType().equals(MetadataIndexNodeType.LEAF_MEASUREMENT)) {
- metadataIndexNode = MetadataIndexNode.deserializeFrom(buffer);
+ try {
+ metadataIndexNode = MetadataIndexNode.deserializeFrom(buffer);
+ } catch (BufferOverflowException e) {
+ logger.error("Something error happened while deserializing MetadataIndexNode of file {}",
+ file, e);
+ throw e;
+ }
metadataIndexPair = getMetadataAndEndOffset(metadataIndexNode,
path.getMeasurement(), MetadataIndexNodeType.INTERNAL_MEASUREMENT, false);
}
@@ -323,7 +336,13 @@ public class TsFileSequenceReader implements AutoCloseable {
List<TimeseriesMetadata> timeseriesMetadataList = new ArrayList<>();
buffer = readData(metadataIndexPair.left.getOffset(), metadataIndexPair.right);
while (buffer.hasRemaining()) {
- timeseriesMetadataList.add(TimeseriesMetadata.deserializeFrom(buffer));
+ try {
+ timeseriesMetadataList.add(TimeseriesMetadata.deserializeFrom(buffer));
+ } catch (BufferOverflowException e) {
+ logger.error("Something error happened while deserializing TimeseriesMetadata of file {}",
+ file, e);
+ throw e;
+ }
}
// return null if path does not exist in the TsFile
int searchResult = binarySearchInTimeseriesMetadataList(timeseriesMetadataList,
@@ -347,7 +366,13 @@ public class TsFileSequenceReader implements AutoCloseable {
ByteBuffer buffer = readData(metadataIndexPair.left.getOffset(), metadataIndexPair.right);
MetadataIndexNode metadataIndexNode = deviceMetadataIndexNode;
if (!metadataIndexNode.getNodeType().equals(MetadataIndexNodeType.LEAF_MEASUREMENT)) {
- metadataIndexNode = MetadataIndexNode.deserializeFrom(buffer);
+ try {
+ metadataIndexNode = MetadataIndexNode.deserializeFrom(buffer);
+ } catch (BufferOverflowException e) {
+ logger.error("Something error happened while deserializing MetadataIndexNode of file {}",
+ file, e);
+ throw e;
+ }
metadataIndexPair = getMetadataAndEndOffset(metadataIndexNode,
path.getMeasurement(), MetadataIndexNodeType.INTERNAL_MEASUREMENT, false);
}
@@ -357,7 +382,14 @@ public class TsFileSequenceReader implements AutoCloseable {
List<TimeseriesMetadata> timeseriesMetadataList = new ArrayList<>();
buffer = readData(metadataIndexPair.left.getOffset(), metadataIndexPair.right);
while (buffer.hasRemaining()) {
- TimeseriesMetadata timeseriesMetadata = TimeseriesMetadata.deserializeFrom(buffer);
+ TimeseriesMetadata timeseriesMetadata;
+ try {
+ timeseriesMetadata = TimeseriesMetadata.deserializeFrom(buffer);
+ } catch (BufferOverflowException e) {
+ logger.error("Something error happened while deserializing TimeseriesMetadata of file {}",
+ file, e);
+ throw e;
+ }
if (allSensors.contains(timeseriesMetadata.getMeasurementId())) {
timeseriesMetadataList.add(timeseriesMetadata);
}
@@ -386,7 +418,13 @@ public class TsFileSequenceReader implements AutoCloseable {
List<TimeseriesMetadata> timeseriesMetadataList = new ArrayList<>();
MetadataIndexNode metadataIndexNode = deviceMetadataIndexNode;
if (!metadataIndexNode.getNodeType().equals(MetadataIndexNodeType.LEAF_MEASUREMENT)) {
- metadataIndexNode = MetadataIndexNode.deserializeFrom(buffer);
+ try {
+ metadataIndexNode = MetadataIndexNode.deserializeFrom(buffer);
+ } catch (BufferOverflowException e) {
+ logger.error("Something error happened while deserializing MetadataIndexNode of file {}",
+ file, e);
+ throw e;
+ }
measurementMetadataIndexPair = getMetadataAndEndOffset(metadataIndexNode,
measurementList.get(i), MetadataIndexNodeType.INTERNAL_MEASUREMENT, false);
}
@@ -396,7 +434,13 @@ public class TsFileSequenceReader implements AutoCloseable {
buffer = readData(measurementMetadataIndexPair.left.getOffset(),
measurementMetadataIndexPair.right);
while (buffer.hasRemaining()) {
- timeseriesMetadataList.add(TimeseriesMetadata.deserializeFrom(buffer));
+ try {
+ timeseriesMetadataList.add(TimeseriesMetadata.deserializeFrom(buffer));
+ } catch (BufferOverflowException e) {
+ logger.error("Something error happened while deserializing TimeseriesMetadata of file {}",
+ file, e);
+ throw e;
+ }
}
for (int j = i; j < measurementList.size(); j++) {
String current = measurementList.get(j);
@@ -583,32 +627,37 @@ public class TsFileSequenceReader implements AutoCloseable {
private void generateMetadataIndex(MetadataIndexEntry metadataIndex, ByteBuffer buffer,
String deviceId, MetadataIndexNodeType type,
Map<String, List<TimeseriesMetadata>> timeseriesMetadataMap) throws IOException {
- switch (type) {
- case INTERNAL_DEVICE:
- case LEAF_DEVICE:
- case INTERNAL_MEASUREMENT:
- deviceId = metadataIndex.getName();
- MetadataIndexNode metadataIndexNode = MetadataIndexNode.deserializeFrom(buffer);
- int metadataIndexListSize = metadataIndexNode.getChildren().size();
- for (int i = 0; i < metadataIndexListSize; i++) {
- long endOffset = metadataIndexNode.getEndOffset();
- if (i != metadataIndexListSize - 1) {
- endOffset = metadataIndexNode.getChildren().get(i + 1).getOffset();
+ try {
+ switch (type) {
+ case INTERNAL_DEVICE:
+ case LEAF_DEVICE:
+ case INTERNAL_MEASUREMENT:
+ deviceId = metadataIndex.getName();
+ MetadataIndexNode metadataIndexNode = MetadataIndexNode.deserializeFrom(buffer);
+ int metadataIndexListSize = metadataIndexNode.getChildren().size();
+ for (int i = 0; i < metadataIndexListSize; i++) {
+ long endOffset = metadataIndexNode.getEndOffset();
+ if (i != metadataIndexListSize - 1) {
+ endOffset = metadataIndexNode.getChildren().get(i + 1).getOffset();
+ }
+ ByteBuffer nextBuffer = readData(metadataIndexNode.getChildren().get(i).getOffset(),
+ endOffset);
+ generateMetadataIndex(metadataIndexNode.getChildren().get(i), nextBuffer, deviceId,
+ metadataIndexNode.getNodeType(), timeseriesMetadataMap);
}
- ByteBuffer nextBuffer = readData(metadataIndexNode.getChildren().get(i).getOffset(),
- endOffset);
- generateMetadataIndex(metadataIndexNode.getChildren().get(i), nextBuffer, deviceId,
- metadataIndexNode.getNodeType(), timeseriesMetadataMap);
- }
- break;
- case LEAF_MEASUREMENT:
- List<TimeseriesMetadata> timeseriesMetadataList = new ArrayList<>();
- while (buffer.hasRemaining()) {
- timeseriesMetadataList.add(TimeseriesMetadata.deserializeFrom(buffer));
- }
- timeseriesMetadataMap.computeIfAbsent(deviceId, k -> new ArrayList<>())
- .addAll(timeseriesMetadataList);
- break;
+ break;
+ case LEAF_MEASUREMENT:
+ List<TimeseriesMetadata> timeseriesMetadataList = new ArrayList<>();
+ while (buffer.hasRemaining()) {
+ timeseriesMetadataList.add(TimeseriesMetadata.deserializeFrom(buffer));
+ }
+ timeseriesMetadataMap.computeIfAbsent(deviceId, k -> new ArrayList<>())
+ .addAll(timeseriesMetadataList);
+ break;
+ }
+ } catch (BufferOverflowException e) {
+ logger.error("Something error happened while generating MetadataIndex of file {}", file, e);
+ throw e;
}
}
@@ -668,14 +717,20 @@ public class TsFileSequenceReader implements AutoCloseable {
*/
private Pair<MetadataIndexEntry, Long> getMetadataAndEndOffset(MetadataIndexNode metadataIndex,
String name, MetadataIndexNodeType type, boolean exactSearch) throws IOException {
- if (!metadataIndex.getNodeType().equals(type)) {
- return metadataIndex.getChildIndexEntry(name, exactSearch);
- } else {
- Pair<MetadataIndexEntry, Long> childIndexEntry = metadataIndex
- .getChildIndexEntry(name, false);
- ByteBuffer buffer = readData(childIndexEntry.left.getOffset(), childIndexEntry.right);
- return getMetadataAndEndOffset(MetadataIndexNode.deserializeFrom(buffer), name, type,
- false);
+ try {
+ if (!metadataIndex.getNodeType().equals(type)) {
+ return metadataIndex.getChildIndexEntry(name, exactSearch);
+ } else {
+ Pair<MetadataIndexEntry, Long> childIndexEntry = metadataIndex
+ .getChildIndexEntry(name, false);
+ ByteBuffer buffer = readData(childIndexEntry.left.getOffset(), childIndexEntry.right);
+ return getMetadataAndEndOffset(MetadataIndexNode.deserializeFrom(buffer), name, type,
+ false);
+ }
+ } catch (BufferOverflowException e) {
+ logger
+ .error("Something error happened while deserializing MetadataIndex of file {}", file, e);
+ throw e;
}
}
@@ -1060,22 +1115,27 @@ public class TsFileSequenceReader implements AutoCloseable {
*/
public List<ChunkMetadata> readChunkMetaDataList(TimeseriesMetadata timeseriesMetaData)
throws IOException {
- readFileMetadata();
- List<Pair<Long, Long>> versionInfo = tsFileMetaData.getVersionInfo();
- ArrayList<ChunkMetadata> chunkMetadataList = new ArrayList<>();
- long startOffsetOfChunkMetadataList = timeseriesMetaData.getOffsetOfChunkMetaDataList();
- int dataSizeOfChunkMetadataList = timeseriesMetaData.getDataSizeOfChunkMetaDataList();
+ try {
+ readFileMetadata();
+ List<Pair<Long, Long>> versionInfo = tsFileMetaData.getVersionInfo();
+ ArrayList<ChunkMetadata> chunkMetadataList = new ArrayList<>();
+ long startOffsetOfChunkMetadataList = timeseriesMetaData.getOffsetOfChunkMetaDataList();
+ int dataSizeOfChunkMetadataList = timeseriesMetaData.getDataSizeOfChunkMetaDataList();
- ByteBuffer buffer = readData(startOffsetOfChunkMetadataList, dataSizeOfChunkMetadataList);
- while (buffer.hasRemaining()) {
- chunkMetadataList.add(ChunkMetadata.deserializeFrom(buffer));
- }
+ ByteBuffer buffer = readData(startOffsetOfChunkMetadataList, dataSizeOfChunkMetadataList);
+ while (buffer.hasRemaining()) {
+ chunkMetadataList.add(ChunkMetadata.deserializeFrom(buffer));
+ }
- VersionUtils.applyVersion(chunkMetadataList, versionInfo);
+ VersionUtils.applyVersion(chunkMetadataList, versionInfo);
- // minimize the storage of an ArrayList instance.
- chunkMetadataList.trimToSize();
- return chunkMetadataList;
+ // minimize the storage of an ArrayList instance.
+ chunkMetadataList.trimToSize();
+ return chunkMetadataList;
+ } catch (BufferOverflowException e) {
+ logger.error("Something error happened while reading ChunkMetaDataList of file {}", file, e);
+ throw e;
+ }
}
/**