diff --git a/tubemq-server/src/main/java/org/apache/tubemq/server/broker/msgstore/disk/FileSegment.java b/tubemq-server/src/main/java/org/apache/tubemq/server/broker/msgstore/disk/FileSegment.java index 949d149faf5..d85c651231c 100644 --- a/tubemq-server/src/main/java/org/apache/tubemq/server/broker/msgstore/disk/FileSegment.java +++ b/tubemq-server/src/main/java/org/apache/tubemq/server/broker/msgstore/disk/FileSegment.java @@ -25,6 +25,7 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicLong; import org.apache.tubemq.corebase.utils.CheckSum; +import org.apache.tubemq.corebase.utils.ServiceStatusHolder; import org.apache.tubemq.server.broker.utils.DataStoreUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -112,6 +113,9 @@ private FileSegment(final long start, final File file, this.cachedSize.set(this.channel.size()); this.flushedSize.set(this.cachedSize.get()); } catch (final Exception e) { + if (e instanceof IOException) { + ServiceStatusHolder.addReadIOErrCnt(); + } if (this.segmentType == SegmentType.DATA) { logger.error("[File Store] Set DATA Segment cachedSize error", e); } else { @@ -133,6 +137,9 @@ public void close() { } this.randFile.close(); } catch (Throwable ee) { + if (ee instanceof IOException) { + ServiceStatusHolder.addReadIOErrCnt(); + } logger.error(new StringBuilder(512).append("[File Store] Close ") .append(this.file.getAbsoluteFile().toString()) .append("'s ").append(segmentType).append(" file failure").toString(), ee); @@ -152,6 +159,9 @@ public void deleteFile() { } this.randFile.close(); } catch (Throwable e1) { + if (e1 instanceof IOException) { + ServiceStatusHolder.addReadIOErrCnt(); + } logger.error("[File Store] failure to close channel ", e1); } try { @@ -160,6 +170,9 @@ public void deleteFile() { .append(file.getAbsoluteFile()).toString()); this.file.delete(); } catch (Throwable ee) { + if (ee instanceof IOException) { + ServiceStatusHolder.addReadIOErrCnt(); + } logger.error("[File Store] failure to delete file ", ee); } }