HBASE-25098 ReplicationStatisticsChore runs in wrong time unit (#2460)

Signed-off-by: Viraj Jasani <vjasani@apache.org>
Signed-off-by: Guanghao Zhang <zghao@apache.org>
This commit is contained in:
XinSun 2020-09-26 19:49:02 +08:00 committed by GitHub
parent 1093e34967
commit 380585ef20
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
2 changed files with 11 additions and 6 deletions

View File

@ -20,6 +20,7 @@ package org.apache.hadoop.hbase.replication;
import java.io.IOException; import java.io.IOException;
import java.util.Collections; import java.util.Collections;
import java.util.List; import java.util.List;
import java.util.concurrent.TimeUnit;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileSystem;
@ -51,7 +52,7 @@ public class ReplicationSinkServiceImpl implements ReplicationSinkService {
// ReplicationLoad to access replication metrics // ReplicationLoad to access replication metrics
private ReplicationLoad replicationLoad; private ReplicationLoad replicationLoad;
private int statsPeriod; private int statsPeriodInSecond;
@Override @Override
public void replicateLogEntries(List<AdminProtos.WALEntry> entries, CellScanner cells, public void replicateLogEntries(List<AdminProtos.WALEntry> entries, CellScanner cells,
@ -66,7 +67,7 @@ public class ReplicationSinkServiceImpl implements ReplicationSinkService {
WALProvider walProvider) throws IOException { WALProvider walProvider) throws IOException {
this.server = server; this.server = server;
this.conf = server.getConfiguration(); this.conf = server.getConfiguration();
this.statsPeriod = this.statsPeriodInSecond =
this.conf.getInt("replication.stats.thread.period.seconds", 5 * 60); this.conf.getInt("replication.stats.thread.period.seconds", 5 * 60);
this.replicationLoad = new ReplicationLoad(); this.replicationLoad = new ReplicationLoad();
} }
@ -75,7 +76,8 @@ public class ReplicationSinkServiceImpl implements ReplicationSinkService {
public void startReplicationService() throws IOException { public void startReplicationService() throws IOException {
this.replicationSink = new ReplicationSink(this.conf); this.replicationSink = new ReplicationSink(this.conf);
this.server.getChoreService().scheduleChore( this.server.getChoreService().scheduleChore(
new ReplicationStatisticsChore("ReplicationSinkStatistics", server, statsPeriod)); new ReplicationStatisticsChore("ReplicationSinkStatistics", server,
(int) TimeUnit.SECONDS.toMillis(statsPeriodInSecond)));
} }
@Override @Override

View File

@ -22,6 +22,8 @@ import java.util.ArrayList;
import java.util.List; import java.util.List;
import java.util.OptionalLong; import java.util.OptionalLong;
import java.util.UUID; import java.util.UUID;
import java.util.concurrent.TimeUnit;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
@ -64,7 +66,7 @@ public class Replication implements ReplicationSourceService {
private SyncReplicationPeerInfoProvider syncReplicationPeerInfoProvider; private SyncReplicationPeerInfoProvider syncReplicationPeerInfoProvider;
// Hosting server // Hosting server
private Server server; private Server server;
private int statsPeriod; private int statsPeriodInSecond;
// ReplicationLoad to access replication metrics // ReplicationLoad to access replication metrics
private ReplicationLoad replicationLoad; private ReplicationLoad replicationLoad;
private MetricsReplicationGlobalSourceSource globalMetricsSource; private MetricsReplicationGlobalSourceSource globalMetricsSource;
@ -139,7 +141,7 @@ public class Replication implements ReplicationSourceService {
p.getSyncReplicationState(), p.getNewSyncReplicationState(), 0)); p.getSyncReplicationState(), p.getNewSyncReplicationState(), 0));
} }
} }
this.statsPeriod = this.statsPeriodInSecond =
this.conf.getInt("replication.stats.thread.period.seconds", 5 * 60); this.conf.getInt("replication.stats.thread.period.seconds", 5 * 60);
this.replicationLoad = new ReplicationLoad(); this.replicationLoad = new ReplicationLoad();
@ -169,7 +171,8 @@ public class Replication implements ReplicationSourceService {
public void startReplicationService() throws IOException { public void startReplicationService() throws IOException {
this.replicationManager.init(); this.replicationManager.init();
this.server.getChoreService().scheduleChore( this.server.getChoreService().scheduleChore(
new ReplicationStatisticsChore("ReplicationSourceStatistics", server, statsPeriod)); new ReplicationStatisticsChore("ReplicationSourceStatistics", server,
(int) TimeUnit.SECONDS.toMillis(statsPeriodInSecond)));
LOG.info("{} started", this.server.toString()); LOG.info("{} started", this.server.toString());
} }