HDFS-12333. Ozone: Extend Datanode web interface with SCM information. Contributed by Elek, Marton.

This commit is contained in:
Anu Engineer 2017-08-29 14:05:28 -07:00 committed by Owen O'Malley
parent ca72c2342c
commit 2e2e30373c
9 changed files with 226 additions and 10 deletions

View File

@ -0,0 +1,36 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.ozone.container.common.interfaces;
import org.apache.hadoop.ozone.container.common.impl.StorageLocationReport;
import java.io.IOException;
/**
* Returns physical path locations, where the containers will be created.
*/
public interface ContainerLocationManagerMXBean {
/**
* Returns an array of storage location usage report.
*
* @return storage location usage report.
*/
StorageLocationReport[] getLocationReport() throws IOException;
}

View File

@ -28,6 +28,7 @@ import org.slf4j.LoggerFactory;
import java.io.Closeable; import java.io.Closeable;
import java.io.IOException; import java.io.IOException;
import java.net.InetSocketAddress; import java.net.InetSocketAddress;
import java.time.ZonedDateTime;
import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicLong;
import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.Lock;
import java.util.concurrent.locks.ReentrantLock; import java.util.concurrent.locks.ReentrantLock;
@ -35,7 +36,8 @@ import java.util.concurrent.locks.ReentrantLock;
/** /**
* Endpoint is used as holder class that keeps state around the RPC endpoint. * Endpoint is used as holder class that keeps state around the RPC endpoint.
*/ */
public class EndpointStateMachine implements Closeable { public class EndpointStateMachine
implements Closeable, EndpointStateMachineMBean {
static final Logger static final Logger
LOG = LoggerFactory.getLogger(EndpointStateMachine.class); LOG = LoggerFactory.getLogger(EndpointStateMachine.class);
private final StorageContainerDatanodeProtocolClientSideTranslatorPB endPoint; private final StorageContainerDatanodeProtocolClientSideTranslatorPB endPoint;
@ -45,6 +47,7 @@ public class EndpointStateMachine implements Closeable {
private final Configuration conf; private final Configuration conf;
private EndPointStates state; private EndPointStates state;
private VersionResponse version; private VersionResponse version;
private ZonedDateTime lastSuccessfulHeartbeat;
/** /**
* Constructs RPC Endpoints. * Constructs RPC Endpoints.
@ -104,6 +107,15 @@ public class EndpointStateMachine implements Closeable {
return state; return state;
} }
@Override
public int getVersionNumber() {
if (version != null) {
return version.getProtobufMessage().getSoftwareVersion();
} else {
return -1;
}
}
/** /**
* Sets the endpoint state. * Sets the endpoint state.
* *
@ -144,6 +156,11 @@ public class EndpointStateMachine implements Closeable {
return this.missedCount.get(); return this.missedCount.get();
} }
@Override
public String getAddressString() {
return getAddress().toString();
}
public void zeroMissedCount() { public void zeroMissedCount() {
this.missedCount.set(0); this.missedCount.set(0);
} }
@ -262,4 +279,15 @@ public class EndpointStateMachine implements Closeable {
return getLastState(); return getLastState();
} }
} }
public long getLastSuccessfulHeartbeat() {
return lastSuccessfulHeartbeat == null ?
0 :
lastSuccessfulHeartbeat.toEpochSecond();
}
public void setLastSuccessfulHeartbeat(
ZonedDateTime lastSuccessfulHeartbeat) {
this.lastSuccessfulHeartbeat = lastSuccessfulHeartbeat;
}
} }

View File

@ -0,0 +1,34 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with this
* work for additional information regarding copyright ownership. The ASF
* licenses this file to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
* <p>
* http://www.apache.org/licenses/LICENSE-2.0
* <p>
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
* WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
* License for the specific language governing permissions and limitations under
* the License.
*/
package org.apache.hadoop.ozone.container.common.statemachine;
/**
* JMX representation of an EndpointStateMachine.
*/
public interface EndpointStateMachineMBean {
long getMissedCount();
String getAddressString();
EndpointStateMachine.EndPointStates getState();
int getVersionNumber();
long getLastSuccessfulHeartbeat();
}

View File

@ -20,6 +20,7 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.ipc.ProtobufRpcEngine; import org.apache.hadoop.ipc.ProtobufRpcEngine;
import org.apache.hadoop.ipc.RPC; import org.apache.hadoop.ipc.RPC;
import org.apache.hadoop.metrics2.util.MBeans;
import org.apache.hadoop.net.NetUtils; import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.ozone.client.OzoneClientUtils; import org.apache.hadoop.ozone.client.OzoneClientUtils;
import org.apache.hadoop.ozone.protocolPB import org.apache.hadoop.ozone.protocolPB
@ -29,12 +30,11 @@ import org.apache.hadoop.security.UserGroupInformation;
import org.slf4j.Logger; import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
import javax.management.ObjectName;
import java.io.Closeable; import java.io.Closeable;
import java.io.IOException; import java.io.IOException;
import java.net.InetSocketAddress; import java.net.InetSocketAddress;
import java.util.Collection; import java.util.*;
import java.util.HashMap;
import java.util.Map;
import java.util.concurrent.locks.ReadWriteLock; import java.util.concurrent.locks.ReadWriteLock;
import java.util.concurrent.locks.ReentrantReadWriteLock; import java.util.concurrent.locks.ReentrantReadWriteLock;
@ -42,7 +42,8 @@ import java.util.concurrent.locks.ReentrantReadWriteLock;
* SCMConnectionManager - Acts as a class that manages the membership * SCMConnectionManager - Acts as a class that manages the membership
* information of the SCMs that we are working with. * information of the SCMs that we are working with.
*/ */
public class SCMConnectionManager implements Closeable{ public class SCMConnectionManager
implements Closeable, SCMConnectionManagerMXBean {
private static final Logger LOG = private static final Logger LOG =
LoggerFactory.getLogger(SCMConnectionManager.class); LoggerFactory.getLogger(SCMConnectionManager.class);
@ -51,7 +52,7 @@ public class SCMConnectionManager implements Closeable{
private final int rpcTimeout; private final int rpcTimeout;
private final Configuration conf; private final Configuration conf;
private final ObjectName jmxBean;
public SCMConnectionManager(Configuration conf) { public SCMConnectionManager(Configuration conf) {
this.mapLock = new ReentrantReadWriteLock(); this.mapLock = new ReentrantReadWriteLock();
@ -59,8 +60,12 @@ public class SCMConnectionManager implements Closeable{
this.rpcTimeout = timeOut.intValue(); this.rpcTimeout = timeOut.intValue();
this.scmMachines = new HashMap<>(); this.scmMachines = new HashMap<>();
this.conf = conf; this.conf = conf;
jmxBean = MBeans.register("OzoneDataNode",
"SCMConnectionManager",
this);
} }
/** /**
* Returns Config. * Returns Config.
* *
@ -179,5 +184,18 @@ public class SCMConnectionManager implements Closeable{
public void close() throws IOException { public void close() throws IOException {
getValues().forEach(endpointStateMachine getValues().forEach(endpointStateMachine
-> IOUtils.cleanupWithLogger(LOG, endpointStateMachine)); -> IOUtils.cleanupWithLogger(LOG, endpointStateMachine));
MBeans.unregister(jmxBean);
}
@Override
public List<EndpointStateMachineMBean> getSCMServers() {
readLock();
try {
return Collections
.unmodifiableList(new ArrayList<>(scmMachines.values()));
} finally {
readUnlock();
}
} }
} }

View File

@ -0,0 +1,27 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with this
* work for additional information regarding copyright ownership. The ASF
* licenses this file to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
* <p>
* http://www.apache.org/licenses/LICENSE-2.0
* <p>
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
* WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
* License for the specific language governing permissions and limitations under
* the License.
*/
package org.apache.hadoop.ozone.container.common.statemachine;
import java.util.List;
/**
* JMX information about the connected SCM servers.
*/
public interface SCMConnectionManagerMXBean {
List<EndpointStateMachineMBean> getSCMServers();
}

View File

@ -40,6 +40,7 @@ import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
import java.io.IOException; import java.io.IOException;
import java.time.ZonedDateTime;
import java.util.concurrent.Callable; import java.util.concurrent.Callable;
/** /**
@ -103,6 +104,7 @@ public class HeartbeatEndpointTask
.sendHeartbeat(datanodeID, this.context.getNodeReport(), .sendHeartbeat(datanodeID, this.context.getNodeReport(),
this.context.getContainerReportState()); this.context.getContainerReportState());
processResponse(reponse); processResponse(reponse);
rpcEndpoint.setLastSuccessfulHeartbeat(ZonedDateTime.now());
rpcEndpoint.zeroMissedCount(); rpcEndpoint.zeroMissedCount();
} catch (IOException ex) { } catch (IOException ex) {
rpcEndpoint.logIfNeeded(ex); rpcEndpoint.logIfNeeded(ex);

View File

@ -95,6 +95,52 @@
{/dn.BPServiceActorInfo} {/dn.BPServiceActorInfo}
</table> </table>
{#ozone.enabled}
<div class="page-header"><h1>Ozone: SCM Connections</h1></div>
<table class="table">
<thead>
<tr>
<th>SCM Address</th>
<th>Status</th>
<th>Version</th>
<th>Missed count</th>
<th>Last heartbeat</th>
</tr>
</thead>
{#ozone.SCMServers}
<tr>
<td>{addressString}</td>
<td>{state}</td>
<td>{versionNumber}</td>
<td>{missedCount}s</td>
<td>{lastSuccessfulHeartbeat|elapsed|fmt_time}</td>
</tr>
{/ozone.SCMServers}
</table>
<div class="page-header"><h1>Ozone: Storage locations</h1></div>
<table class="table">
<thead>
<tr>
<th>ID</th>
<th>Capacity</th>
<th>Remaining</th>
<th>SCM used</th>
<th>failed</th>
</tr>
</thead>
{#ozone.LocationReport}
<tr>
<td>{id}</td>
<td>{capacity|fmt_bytes}</td>
<td>{remaining|fmt_bytes}</td>
<td>{scmUsed|fmt_bytes}</td>
<td>{failed}</td>
</tr>
{/ozone.LocationReport}
</table>
{/ozone.enabled}
<div class="page-header"><h1>Volume Information</h1></div> <div class="page-header"><h1>Volume Information</h1></div>
<table class="table"> <table class="table">
<thead> <thead>

View File

@ -18,11 +18,11 @@
(function () { (function () {
"use strict"; "use strict";
var data = {}; var data = {ozone: {enabled: false}};
dust.loadSource(dust.compile($('#tmpl-dn').html(), 'dn')); dust.loadSource(dust.compile($('#tmpl-dn').html(), 'dn'));
function load() { function loadDatanodeInfo() {
$.get('/jmx?qry=Hadoop:service=DataNode,name=DataNodeInfo', function(resp) { $.get('/jmx?qry=Hadoop:service=DataNode,name=DataNodeInfo', function(resp) {
data.dn = workaround(resp.beans[0]); data.dn = workaround(resp.beans[0]);
data.dn.HostName = resp.beans[0]['DatanodeHostname']; data.dn.HostName = resp.beans[0]['DatanodeHostname'];
@ -30,6 +30,26 @@
}).fail(show_err_msg); }).fail(show_err_msg);
} }
function loadOzoneScmInfo() {
$.get('/jmx?qry=Hadoop:service=OzoneDataNode,name=SCMConnectionManager', function (resp) {
if (resp.beans.length > 0) {
data.ozone.SCMServers = resp.beans[0].SCMServers;
data.ozone.enabled = true;
render();
}
}).fail(show_err_msg);
}
function loadOzoneStorageInfo() {
$.get('/jmx?qry=Hadoop:service=OzoneDataNode,name=ContainerLocationManager', function (resp) {
if (resp.beans.length > 0) {
data.ozone.LocationReport = resp.beans[0].LocationReport;
data.ozone.enabled = true;
render();
}
}).fail(show_err_msg);
}
function workaround(dn) { function workaround(dn) {
function node_map_to_array(nodes) { function node_map_to_array(nodes) {
var res = []; var res = [];
@ -65,6 +85,8 @@
$('#alert-panel').show(); $('#alert-panel').show();
} }
load(); loadDatanodeInfo();
loadOzoneScmInfo();
loadOzoneStorageInfo();
})(); })();

View File

@ -38,7 +38,10 @@
'fmt_percentage': function (v) { 'fmt_percentage': function (v) {
return Math.round(v * 100) / 100 + '%'; return Math.round(v * 100) / 100 + '%';
}, },
'elapsed': function(v) {
//elapsed sec from epoch sec
return Date.now() - v * 1000;
},
'fmt_time': function (v) { 'fmt_time': function (v) {
var s = Math.floor(v / 1000), h = Math.floor(s / 3600); var s = Math.floor(v / 1000), h = Math.floor(s / 3600);
s -= h * 3600; s -= h * 3600;