diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java index 04a54e0cf23..8dd856875c3 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java @@ -325,11 +325,16 @@ public class ReplicationSource implements ReplicationSourceInterface { replicationDelay = ReplicationLoad.calculateReplicationDelay(ageOfLastShippedOp, lastTimeStamp, queueSize); Path currentPath = shipper.getCurrentPath(); - try { - fileSize = getFileSize(currentPath); - } catch (IOException e) { - LOG.warn("Ignore the exception as the file size of HLog only affects the web ui", e); - fileSize = -1; + fileSize = -1; + if (currentPath != null) { + try { + fileSize = getFileSize(currentPath); + } catch (IOException e) { + LOG.warn("Ignore the exception as the file size of HLog only affects the web ui", e); + } + } else { + currentPath = new Path("NO_LOGS_IN_QUEUE"); + LOG.warn("No replication ongoing, waiting for new log"); } ReplicationStatus.ReplicationStatusBuilder statusBuilder = ReplicationStatus.newBuilder(); statusBuilder.withPeerId(this.getPeerId())