Skip to content

HBASE-28631 Show the create time of the replication peer #5954

New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Open
wants to merge 1 commit into
base: master
Choose a base branch
from
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -32,6 +32,7 @@ public class ReplicationPeerDescription {
private final boolean enabled;
private final ReplicationPeerConfig config;
private final SyncReplicationState syncReplicationState;
private String createTime;

public ReplicationPeerDescription(String id, boolean enabled, ReplicationPeerConfig config,
SyncReplicationState syncReplicationState) {
Expand All @@ -41,6 +42,19 @@ public ReplicationPeerDescription(String id, boolean enabled, ReplicationPeerCon
this.syncReplicationState = syncReplicationState;
}

public ReplicationPeerDescription(String id, boolean enabled, ReplicationPeerConfig config,
SyncReplicationState syncReplicationState, String createTime) {
this.id = id;
this.enabled = enabled;
this.config = config;
this.syncReplicationState = syncReplicationState;
this.createTime = createTime;
}

public String getCreateTime() {
return createTime;
}

public String getPeerId() {
return this.id;
}
Expand All @@ -63,6 +77,7 @@ public String toString() {
builder.append(", enabled : " + enabled);
builder.append(", config : " + config);
builder.append(", syncReplicationState : " + syncReplicationState);
builder.append(", createTime : " + createTime);
return builder.toString();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -77,6 +77,8 @@ public class FSReplicationPeerStorage implements ReplicationPeerStorage {

static final String SYNC_REPLICATION_STATE_FILE = "sync-rep-state";

static final String CREATE_TIME_FILE = "create_time";

static final byte[] NONE_STATE_BYTES =
SyncReplicationState.toByteArray(SyncReplicationState.NONE);

Expand Down Expand Up @@ -111,6 +113,7 @@ public void addPeer(String peerId, ReplicationPeerConfig peerConfig, boolean ena
if (!enabled) {
fs.createNewFile(new Path(peerDir, DISABLED_FILE));
}
fs.createNewFile(new Path(peerDir, CREATE_TIME_FILE));
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I have changed it to just create this file and not add content. So I can use fs.getFileStatus(createTimeFile).getModificationTime() to get peer create time. @Apache9

write(fs, peerDir, SYNC_REPLICATION_STATE_FILE,
SyncReplicationState.toByteArray(syncReplicationState, SyncReplicationState.NONE));
// write the peer config data at last, so when loading, if we can not load the peer_config, we
Expand Down Expand Up @@ -227,6 +230,19 @@ public ReplicationPeerConfig getPeerConfig(String peerId) throws ReplicationExce
}
}

@Override
public long getPeerCreateTime(String peerId) {
Path createTimeFile = new Path(getPeerDir(peerId), CREATE_TIME_FILE);
try {
if (fs.exists(createTimeFile)) {
return fs.getFileStatus(createTimeFile).getModificationTime();
}
} catch (IOException e) {
LOG.warn("Unable to get create time of the peer: " + peerId, e);
}
return NO_CREATE_TIME;
}

private Pair<SyncReplicationState, SyncReplicationState> getStateAndNewState(String peerId)
throws IOException {
Path peerDir = getPeerDir(peerId);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,11 @@
@InterfaceAudience.Private
public interface ReplicationPeerStorage {

/**
* When the peer has no creation time, this constant is set to 1 by default
*/
long NO_CREATE_TIME = -1;

/**
* Add a replication peer.
* @throws ReplicationException if there are errors accessing the storage service.
Expand Down Expand Up @@ -70,6 +75,11 @@ void updatePeerConfig(String peerId, ReplicationPeerConfig peerConfig)
*/
ReplicationPeerConfig getPeerConfig(String peerId) throws ReplicationException;

/**
* Get the peer create time of a replication peer.
*/
long getPeerCreateTime(String peerId);

/**
* Set the new sync replication state that we are going to transit to.
* @throws ReplicationException if there are errors accessing the storage service.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -32,6 +32,7 @@
import org.apache.zookeeper.KeeperException;

import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos;
import org.apache.zookeeper.data.Stat;

/**
* ZK based replication peer storage.
Expand Down Expand Up @@ -185,6 +186,16 @@ public ReplicationPeerConfig getPeerConfig(String peerId) throws ReplicationExce
}
}

@Override
public long getPeerCreateTime(String peerId) {
Stat createTimeIfNodeExists = ZKUtil.getCreateTimeIfNodeExists(zookeeper, getPeerNode(peerId));
if (createTimeIfNodeExists == null) {
return NO_CREATE_TIME;
} else {
return createTimeIfNodeExists.getCtime();
}
}

@Override
public void setPeerNewSyncReplicationState(String peerId, SyncReplicationState state)
throws ReplicationException {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -750,6 +750,7 @@ AssignmentManager assignmentManager = master.getAssignmentManager();
</%java>
<table class="table table-striped">
<tr>
<th>Create Time</th>
<th>Peer Id</th>
<th>Cluster Key</th>
<th>Endpoint</th>
Expand All @@ -771,6 +772,7 @@ AssignmentManager assignmentManager = master.getAssignmentManager();
ReplicationPeerConfig peerConfig = peer.getPeerConfig();
</%java>
<tr>
<td><% peer.getCreateTime() %></td>
<td><% peerId %></td>
<td><% peerConfig.getClusterKey() %></td>
<td><% peerConfig.getReplicationEndpointImpl() %></td>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,8 @@
import com.google.errorprone.annotations.RestrictedApi;
import java.io.IOException;
import java.net.URI;
import java.text.DateFormat;
import java.text.SimpleDateFormat;
import java.util.ArrayList;
import java.util.Collection;
import java.util.EnumSet;
Expand Down Expand Up @@ -96,6 +98,8 @@ public class ReplicationPeerManager implements ConfigurationObserver {

private static final Logger LOG = LoggerFactory.getLogger(ReplicationPeerManager.class);

private static final DateFormat DATE_FORMAT = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss");

private volatile ReplicationPeerStorage peerStorage;

private final ReplicationQueueStorage queueStorage;
Expand Down Expand Up @@ -288,8 +292,10 @@ public void addPeer(String peerId, ReplicationPeerConfig peerConfig, boolean ena
? SyncReplicationState.DOWNGRADE_ACTIVE
: SyncReplicationState.NONE;
peerStorage.addPeer(peerId, copiedPeerConfig, enabled, syncReplicationState);
long peerCreateTime = peerStorage.getPeerCreateTime(peerId);
String peerCreateTimeStr = peerCreateTime == -1 ? "" : DATE_FORMAT.format(peerCreateTime);
peers.put(peerId,
new ReplicationPeerDescription(peerId, enabled, copiedPeerConfig, syncReplicationState));
new ReplicationPeerDescription(peerId, enabled, copiedPeerConfig, syncReplicationState, peerCreateTimeStr));
}

public void removePeer(String peerId) throws ReplicationException {
Expand All @@ -309,7 +315,7 @@ private void setPeerState(String peerId, boolean enabled) throws ReplicationExce
}
peerStorage.setPeerState(peerId, enabled);
peers.put(peerId, new ReplicationPeerDescription(peerId, enabled, desc.getPeerConfig(),
desc.getSyncReplicationState()));
desc.getSyncReplicationState(), desc.getCreateTime()));
}

public boolean getPeerState(String peerId) throws ReplicationException {
Expand Down Expand Up @@ -342,7 +348,7 @@ public void updatePeerConfig(String peerId, ReplicationPeerConfig peerConfig)
ReplicationPeerConfig newPeerConfig = newPeerConfigBuilder.build();
peerStorage.updatePeerConfig(peerId, newPeerConfig);
peers.put(peerId, new ReplicationPeerDescription(peerId, desc.isEnabled(), newPeerConfig,
desc.getSyncReplicationState()));
desc.getSyncReplicationState(), desc.getCreateTime()));
}

public List<ReplicationPeerDescription> listPeers(Pattern pattern) {
Expand Down Expand Up @@ -377,7 +383,7 @@ public void transitPeerSyncReplicationState(String peerId, SyncReplicationState
if (desc.getSyncReplicationState() != newState) {
// Only recreate the desc if this is not a retry
peers.put(peerId,
new ReplicationPeerDescription(peerId, desc.isEnabled(), desc.getPeerConfig(), newState));
new ReplicationPeerDescription(peerId, desc.isEnabled(), desc.getPeerConfig(), newState, desc.getCreateTime()));
}
}

Expand Down Expand Up @@ -692,7 +698,10 @@ public void run() {
peerStorage.updatePeerConfig(peerId, peerConfig);
boolean enabled = peerStorage.isPeerEnabled(peerId);
SyncReplicationState state = peerStorage.getPeerSyncReplicationState(peerId);
peers.put(peerId, new ReplicationPeerDescription(peerId, enabled, peerConfig, state));
long peerCreateTime = peerStorage.getPeerCreateTime(peerId);
String peerCreateTimeStr = peerCreateTime == -1 ? "" : DATE_FORMAT.format(peerCreateTime);
peers.put(peerId,
new ReplicationPeerDescription(peerId, enabled, peerConfig, state, peerCreateTimeStr));
}
return new ReplicationPeerManager(fs, zk, peerStorage, queueStorage, peers, conf, clusterId,
pair.getSecond());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -207,7 +207,7 @@ public void testLogCleaning() throws Exception {
}
// Case 4: the newest 3 WALs will be kept because they are beyond the replication offset
masterServices.getReplicationPeerManager().listPeers(null)
.add(new ReplicationPeerDescription(peerId, true, null, null));
.add(new ReplicationPeerDescription(peerId, true, null, null, null));
queueStorage.setOffset(new ReplicationQueueId(server.getServerName(), peerId), fakeMachineName,
new ReplicationGroupOffset(fakeMachineName + "." + (now - 3), 0), Collections.emptyMap());
// Case 5: 5 Procedure WALs that are new, will stay
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -123,7 +123,7 @@ private static FileStatus createFileStatus(ServerName sn, int number) {
}

private static ReplicationPeerDescription createPeer(String peerId) {
return new ReplicationPeerDescription(peerId, true, null, null);
return new ReplicationPeerDescription(peerId, true, null, null, null);
}

private void addServer(ServerName serverName) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -168,6 +168,25 @@ public static int checkExists(ZKWatcher zkw, String znode) throws KeeperExceptio
}
}

/**
* Get the create time if the specified node exists.
* @param zkw zk reference
* @param znode path of node to watch
* @return A Stat instance
* @throws KeeperException if unexpected zookeeper exception
*/
public static Stat getCreateTimeIfNodeExists(ZKWatcher zkw, String znode) {
try {
return zkw.getRecoverableZooKeeper().exists(znode, false);
} catch (KeeperException e) {
LOG.warn(zkw.prefix("Unable to get create time on znode (" + znode + ")"), e);
return null;
} catch (InterruptedException e) {
LOG.warn(zkw.prefix("Unable to get create time on znode (" + znode + ")"), e);
return null;
}
}

//
// Znode listings
//
Expand Down