HDFS-12869. Ozone: Service Discovery: RPC endpoint in KSM for getServiceList. Contributed by Nanda kumar.

This commit is contained in:
Nanda kumar 2017-12-21 14:46:19 +05:30 committed by Owen O'Malley
parent 79432ee6cf
commit 4b9f66a19e
8 changed files with 414 additions and 23 deletions

View File

@ -0,0 +1,191 @@
/**
* 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.ksm.helpers;
import com.google.common.base.Preconditions;
import org.apache.hadoop.ozone.protocol.proto.KeySpaceManagerProtocolProtos;
import org.apache.hadoop.ozone.protocol.proto.KeySpaceManagerProtocolProtos
.ServicePort;
import org.apache.hadoop.ozone.protocol.proto.OzoneProtos.NodeType;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.stream.Collectors;
/**
* ServiceInfo holds the config details of Ozone services.
*/
public final class ServiceInfo {
/**
* Type of node/service.
*/
private final NodeType nodeType;
/**
* Hostname of the node in which the service is running.
*/
private final String hostname;
/**
* List of ports the service listens to.
*/
private final Map<ServicePort.Type, ServicePort> portsMap;
/**
* Constructs the ServiceInfo for the {@code nodeType}.
* @param nodeType type of node/service
* @param hostname hostname of the service
* @param ports list of ports the service listens to
*/
private ServiceInfo(
NodeType nodeType, String hostname, List<ServicePort> ports) {
Preconditions.checkNotNull(nodeType);
Preconditions.checkNotNull(hostname);
this.nodeType = nodeType;
this.hostname = hostname;
this.portsMap = new HashMap<>();
for (ServicePort port : ports) {
portsMap.put(port.getType(), port);
}
}
/**
* Returns the type of node/service.
* @return node type
*/
public NodeType getNodeType() {
return nodeType;
}
/**
* Returns the hostname of the service.
* @return hostname
*/
public String getHostname() {
return hostname;
}
/**
* Returns the list of port which the service listens to.
* @return List<ServicePort>
*/
public List<ServicePort> getPorts() {
return portsMap.values().parallelStream().collect(Collectors.toList());
}
/**
* Returns the port for given type, null if the service doesn't support
* the type.
*
* @param type the type of port.
* ex: RPC, HTTP, HTTPS, etc..
*/
public int getPort(ServicePort.Type type) {
return portsMap.get(type).getValue();
}
/**
* Converts {@link ServiceInfo} to KeySpaceManagerProtocolProtos.ServiceInfo.
*
* @return KeySpaceManagerProtocolProtos.ServiceInfo
*/
public KeySpaceManagerProtocolProtos.ServiceInfo getProtobuf() {
KeySpaceManagerProtocolProtos.ServiceInfo.Builder builder =
KeySpaceManagerProtocolProtos.ServiceInfo.newBuilder();
builder.setNodeType(nodeType)
.setHostname(hostname)
.addAllServicePorts(portsMap.values());
return builder.build();
}
/**
* Converts KeySpaceManagerProtocolProtos.ServiceInfo to {@link ServiceInfo}.
*
* @return {@link ServiceInfo}
*/
public static ServiceInfo getFromProtobuf(
KeySpaceManagerProtocolProtos.ServiceInfo serviceInfo) {
return new ServiceInfo(serviceInfo.getNodeType(),
serviceInfo.getHostname(),
serviceInfo.getServicePortsList());
}
/**
* Creates a new builder to build {@link ServiceInfo}.
* @return {@link ServiceInfo.Builder}
*/
public static Builder newBuilder() {
return new Builder();
}
/**
* Builder used to build/construct {@link ServiceInfo}.
*/
public static class Builder {
private NodeType node;
private String host;
private List<ServicePort> ports = new ArrayList<>();
/**
* Sets the node/service type.
* @param nodeType type of node
* @return the builder
*/
public Builder setNodeType(NodeType nodeType) {
node = nodeType;
return this;
}
/**
* Sets the hostname of the service.
* @param hostname service hostname
* @return the builder
*/
public Builder setHostname(String hostname) {
host = hostname;
return this;
}
/**
* Adds the service port to the service port list.
* @param servicePort RPC port
* @return the builder
*/
public Builder addServicePort(ServicePort servicePort) {
ports.add(servicePort);
return this;
}
/**
* Builds and returns {@link ServiceInfo} with the set values.
* @return {@link ServiceInfo}
*/
public ServiceInfo build() {
return new ServiceInfo(node, host, ports);
}
}
}

View File

@ -24,6 +24,7 @@
import org.apache.hadoop.ozone.ksm.helpers.KsmKeyLocationInfo;
import org.apache.hadoop.ozone.ksm.helpers.KsmVolumeArgs;
import org.apache.hadoop.ozone.ksm.helpers.OpenKeySession;
import org.apache.hadoop.ozone.ksm.helpers.ServiceInfo;
import org.apache.hadoop.ozone.protocol.proto
.KeySpaceManagerProtocolProtos.OzoneAclInfo;
import java.io.IOException;
@ -233,4 +234,12 @@ List<KsmBucketInfo> listBuckets(String volumeName,
List<KsmKeyInfo> listKeys(String volumeName,
String bucketName, String startKeyName, String keyPrefix, int maxKeys)
throws IOException;
/**
* Returns list of Ozone services with its configuration details.
*
* @return list of Ozone services
* @throws IOException
*/
List<ServiceInfo> getServiceList() throws IOException;
}

View File

@ -31,6 +31,7 @@
import org.apache.hadoop.ozone.ksm.helpers.KsmKeyLocationInfo;
import org.apache.hadoop.ozone.ksm.helpers.KsmVolumeArgs;
import org.apache.hadoop.ozone.ksm.helpers.OpenKeySession;
import org.apache.hadoop.ozone.ksm.helpers.ServiceInfo;
import org.apache.hadoop.ozone.ksm.protocol.KeySpaceManagerProtocol;
import org.apache.hadoop.ozone.protocol.proto
.KeySpaceManagerProtocolProtos.AllocateBlockRequest;
@ -90,18 +91,24 @@
.KeySpaceManagerProtocolProtos.ListBucketsRequest;
import org.apache.hadoop.ozone.protocol.proto
.KeySpaceManagerProtocolProtos.ListBucketsResponse;
import org.apache.hadoop.ozone.protocol.proto.KeySpaceManagerProtocolProtos.ListKeysRequest;
import org.apache.hadoop.ozone.protocol.proto.KeySpaceManagerProtocolProtos.ListKeysResponse;
import org.apache.hadoop.ozone.protocol.proto
.KeySpaceManagerProtocolProtos.ListKeysRequest;
import org.apache.hadoop.ozone.protocol.proto
.KeySpaceManagerProtocolProtos.ListKeysResponse;
import org.apache.hadoop.ozone.protocol.proto
.KeySpaceManagerProtocolProtos.VolumeInfo;
import org.apache.hadoop.ozone.protocol.proto
.KeySpaceManagerProtocolProtos.Status;
import org.apache.hadoop.ozone.protocol.proto
.KeySpaceManagerProtocolProtos.OzoneAclInfo;
import org.apache.hadoop.ozone.protocol.proto.KeySpaceManagerProtocolProtos
.ListVolumeRequest;
import org.apache.hadoop.ozone.protocol.proto.KeySpaceManagerProtocolProtos
.ListVolumeResponse;
import org.apache.hadoop.ozone.protocol.proto
.KeySpaceManagerProtocolProtos.ListVolumeRequest;
import org.apache.hadoop.ozone.protocol.proto
.KeySpaceManagerProtocolProtos.ListVolumeResponse;
import org.apache.hadoop.ozone.protocol.proto
.KeySpaceManagerProtocolProtos.ServiceListRequest;
import org.apache.hadoop.ozone.protocol.proto
.KeySpaceManagerProtocolProtos.ServiceListResponse;
import java.io.Closeable;
import java.io.IOException;
@ -705,6 +712,26 @@ public List<KsmKeyInfo> listKeys(String volumeName, String bucketName,
}
}
@Override
public List<ServiceInfo> getServiceList() throws IOException {
ServiceListRequest request = ServiceListRequest.newBuilder().build();
final ServiceListResponse resp;
try {
resp = rpcProxy.getServiceList(NULL_RPC_CONTROLLER, request);
} catch (ServiceException e) {
throw ProtobufHelper.getRemoteException(e);
}
if (resp.getStatus() == Status.OK) {
return resp.getServiceInfoList().stream()
.map(ServiceInfo::getFromProtobuf)
.collect(Collectors.toList());
} else {
throw new IOException("Getting service list failed, error: "
+ resp.getStatus());
}
}
/**
* Return the proxy object underlying this protocol translator.
*

View File

@ -318,6 +318,31 @@ message CommitKeyResponse {
required Status status = 1;
}
message ServiceListRequest {
}
message ServiceListResponse {
required Status status = 1;
repeated ServiceInfo serviceInfo = 2;
}
message ServicePort {
enum Type {
RPC = 1;
HTTP = 2;
HTTPS = 3;
RATIS = 4;
};
required Type type = 1;
required uint32 value = 2;
}
message ServiceInfo {
required NodeType nodeType = 1;
required string hostname = 2;
repeated ServicePort servicePorts = 3;
}
/**
The KSM service that takes care of Ozone namespace.
*/
@ -423,4 +448,10 @@ service KeySpaceManagerService {
*/
rpc allocateBlock(AllocateBlockRequest)
returns(AllocateBlockResponse);
/**
Returns list of Ozone services with its configuration details.
*/
rpc getServiceList(ServiceListRequest)
returns(ServiceListResponse);
}

View File

@ -58,6 +58,7 @@ public class KSMMetrics {
private @Metric MutableCounterLong numVolumeLists;
private @Metric MutableCounterLong numKeyCommits;
private @Metric MutableCounterLong numAllocateBlockCalls;
private @Metric MutableCounterLong numGetServiceLists;
// Failure Metrics
private @Metric MutableCounterLong numVolumeCreateFails;
@ -77,6 +78,7 @@ public class KSMMetrics {
private @Metric MutableCounterLong numVolumeListFails;
private @Metric MutableCounterLong numKeyCommitFails;
private @Metric MutableCounterLong numBlockAllocateCallFails;
private @Metric MutableCounterLong numGetServiceListFails;
public KSMMetrics() {
}
@ -148,6 +150,10 @@ public void incNumVolumeLists() {
numVolumeLists.incr();
}
public void incNumGetServiceLists() {
numGetServiceLists.incr();
}
public void incNumVolumeCreateFails() {
numVolumeCreateFails.incr();
}
@ -240,6 +246,10 @@ public void incNumVolumeListFails() {
numVolumeListFails.incr();
}
public void incNumGetServiceListFails() {
numGetServiceListFails.incr();
}
@VisibleForTesting
public long getNumVolumeCreates() {
return numVolumeCreates.value();
@ -300,6 +310,11 @@ public long getNumKeyLists() {
return numKeyLists.value();
}
@VisibleForTesting
public long getNumGetServiceLists() {
return numGetServiceLists.value();
}
@VisibleForTesting
public long getNumVolumeCreateFails() {
return numVolumeCreateFails.value();
@ -410,6 +425,11 @@ public long getNumBlockAllocateFails() {
return numBlockAllocateCallFails.value();
}
@VisibleForTesting
public long getNumGetServiceListFails() {
return numGetServiceListFails.value();
}
public void unRegister() {
MetricsSystem ms = DefaultMetricsSystem.instance();
ms.unregisterSource(SOURCE_NAME);

View File

@ -18,6 +18,7 @@
package org.apache.hadoop.ozone.ksm;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;
import com.google.protobuf.BlockingService;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hdfs.DFSUtil;
@ -34,6 +35,7 @@
import org.apache.hadoop.ozone.ksm.helpers.KsmKeyLocationInfo;
import org.apache.hadoop.ozone.ksm.helpers.KsmVolumeArgs;
import org.apache.hadoop.ozone.ksm.helpers.OpenKeySession;
import org.apache.hadoop.ozone.ksm.helpers.ServiceInfo;
import org.apache.hadoop.ozone.ksm.protocol.KeySpaceManagerProtocol;
import org.apache.hadoop.ozone.ksm.protocolPB.KeySpaceManagerProtocolPB;
import org.apache.hadoop.ozone.ksm.exceptions.KSMException.ResultCodes;
@ -42,8 +44,11 @@
import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.ozone.client.OzoneClientUtils;
import org.apache.hadoop.conf.OzoneConfiguration;
import org.apache.hadoop.ozone.protocol.proto.KeySpaceManagerProtocolProtos
.ServicePort;
import org.apache.hadoop.ozone.protocol.proto
.KeySpaceManagerProtocolProtos.OzoneAclInfo;
import org.apache.hadoop.ozone.protocol.proto.OzoneProtos;
import org.apache.hadoop.ozone.protocolPB
.KeySpaceManagerProtocolServerSideTranslatorPB;
import org.apache.hadoop.scm.ScmInfo;
@ -60,6 +65,7 @@
import java.io.IOException;
import java.io.PrintStream;
import java.net.InetSocketAddress;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
@ -116,6 +122,7 @@ public static StartupOption parse(String value) {
}
}
private final OzoneConfiguration configuration;
private final RPC.Server ksmRpcServer;
private final InetSocketAddress ksmRpcAddress;
private final KSMMetadataManager metadataManager;
@ -125,11 +132,14 @@ public static StartupOption parse(String value) {
private final KSMMetrics metrics;
private final KeySpaceManagerHttpServer httpServer;
private final KSMStorage ksmStorage;
private final ScmBlockLocationProtocol scmBlockClient;
private ObjectName ksmInfoBeanName;
private KeySpaceManager(OzoneConfiguration conf) throws IOException {
Preconditions.checkNotNull(conf);
configuration = conf;
ksmStorage = new KSMStorage(conf);
ScmBlockLocationProtocol scmBlockClient = getScmBlockClient(conf);
scmBlockClient = getScmBlockClient(configuration);
if (ksmStorage.getState() != StorageState.INITIALIZED) {
throw new KSMException("KSM not initialized.",
ResultCodes.KSM_NOT_INITIALIZED);
@ -145,32 +155,31 @@ private KeySpaceManager(OzoneConfiguration conf) throws IOException {
final int handlerCount = conf.getInt(OZONE_KSM_HANDLER_COUNT_KEY,
OZONE_KSM_HANDLER_COUNT_DEFAULT);
RPC.setProtocolEngine(conf, KeySpaceManagerProtocolPB.class,
RPC.setProtocolEngine(configuration, KeySpaceManagerProtocolPB.class,
ProtobufRpcEngine.class);
BlockingService ksmService = newReflectiveBlockingService(
new KeySpaceManagerProtocolServerSideTranslatorPB(this));
final InetSocketAddress ksmNodeRpcAddr = OzoneClientUtils.
getKsmAddress(conf);
ksmRpcServer = startRpcServer(conf, ksmNodeRpcAddr,
getKsmAddress(configuration);
ksmRpcServer = startRpcServer(configuration, ksmNodeRpcAddr,
KeySpaceManagerProtocolPB.class, ksmService,
handlerCount);
ksmRpcAddress = OzoneClientUtils.updateRPCListenAddress(conf,
ksmRpcAddress = OzoneClientUtils.updateRPCListenAddress(configuration,
OZONE_KSM_ADDRESS_KEY, ksmNodeRpcAddr, ksmRpcServer);
metadataManager = new KSMMetadataManagerImpl(conf);
volumeManager = new VolumeManagerImpl(metadataManager, conf);
metadataManager = new KSMMetadataManagerImpl(configuration);
volumeManager = new VolumeManagerImpl(metadataManager, configuration);
bucketManager = new BucketManagerImpl(metadataManager);
metrics = KSMMetrics.create();
keyManager = new KeyManagerImpl(
getScmBlockClient(conf), metadataManager, conf);
httpServer = new KeySpaceManagerHttpServer(conf);
keyManager = new KeyManagerImpl(scmBlockClient, metadataManager,
configuration);
httpServer = new KeySpaceManagerHttpServer(configuration);
}
/**
* Create a scm block client, used by putKey() and getKey().
*
* @param conf
* @return
* @return {@link ScmBlockLocationProtocol}
* @throws IOException
*/
private static ScmBlockLocationProtocol getScmBlockClient(
@ -191,8 +200,8 @@ private static ScmBlockLocationProtocol getScmBlockClient(
}
@VisibleForTesting
public ScmInfo getScmInfo(OzoneConfiguration conf) throws IOException {
return getScmBlockClient(conf).getScmInfo();
public ScmInfo getScmInfo() throws IOException {
return scmBlockClient.getScmInfo();
}
@VisibleForTesting
@ -775,4 +784,51 @@ private void unregisterMXBean() {
public String getRpcPort() {
return "" + ksmRpcAddress.getPort();
}
@Override
public List<ServiceInfo> getServiceList() throws IOException {
// When we implement multi-home this call has to be handled properly.
List<ServiceInfo> services = new ArrayList<>();
ServiceInfo.Builder ksmServiceInfoBuilder = ServiceInfo.newBuilder()
.setNodeType(OzoneProtos.NodeType.KSM)
.setHostname(ksmRpcAddress.getHostName())
.addServicePort(ServicePort.newBuilder()
.setType(ServicePort.Type.RPC)
.setValue(ksmRpcAddress.getPort())
.build());
if (httpServer.getHttpAddress() != null) {
ksmServiceInfoBuilder.addServicePort(ServicePort.newBuilder()
.setType(ServicePort.Type.HTTP)
.setValue(httpServer.getHttpAddress().getPort())
.build());
}
if (httpServer.getHttpsAddress() != null) {
ksmServiceInfoBuilder.addServicePort(ServicePort.newBuilder()
.setType(ServicePort.Type.HTTPS)
.setValue(httpServer.getHttpsAddress().getPort())
.build());
}
// For client we have to return SCM with container protocol port,
// not block protocol.
InetSocketAddress scmAddr = OzoneClientUtils.getScmAddressForClients(
configuration);
ServiceInfo.Builder scmServiceInfoBuilder = ServiceInfo.newBuilder()
.setNodeType(OzoneProtos.NodeType.SCM)
.setHostname(scmAddr.getHostName())
.addServicePort(ServicePort.newBuilder()
.setType(ServicePort.Type.RPC)
.setValue(scmAddr.getPort()).build());
// TODO: REST servers (datanode) details to be added later.
services.add(ksmServiceInfoBuilder.build());
services.add(scmServiceInfoBuilder.build());
metrics.incNumGetServiceLists();
// For now there is no exception that can can happen in this call,
// so failure metrics is not handled. In future if there is any need to
// handle exception in this method, we need to incorporate
// metrics.incNumGetServiceListFails()
return services;
}
}

View File

@ -26,6 +26,7 @@
import org.apache.hadoop.ozone.ksm.helpers.KsmKeyLocationInfo;
import org.apache.hadoop.ozone.ksm.helpers.KsmVolumeArgs;
import org.apache.hadoop.ozone.ksm.helpers.OpenKeySession;
import org.apache.hadoop.ozone.ksm.helpers.ServiceInfo;
import org.apache.hadoop.ozone.ksm.protocol.KeySpaceManagerProtocol;
import org.apache.hadoop.ozone.ksm.protocolPB.KeySpaceManagerProtocolPB;
import org.apache.hadoop.ozone.ksm.exceptions.KSMException;
@ -87,16 +88,23 @@
.KeySpaceManagerProtocolProtos.ListBucketsRequest;
import org.apache.hadoop.ozone.protocol.proto
.KeySpaceManagerProtocolProtos.ListBucketsResponse;
import org.apache.hadoop.ozone.protocol.proto.KeySpaceManagerProtocolProtos.ListKeysRequest;
import org.apache.hadoop.ozone.protocol.proto.KeySpaceManagerProtocolProtos.ListKeysResponse;
import org.apache.hadoop.ozone.protocol.proto
.KeySpaceManagerProtocolProtos.ListKeysRequest;
import org.apache.hadoop.ozone.protocol.proto
.KeySpaceManagerProtocolProtos.ListKeysResponse;
import org.apache.hadoop.ozone.protocol.proto
.KeySpaceManagerProtocolProtos.Status;
import org.apache.hadoop.ozone.protocol.proto
.KeySpaceManagerProtocolProtos.ServiceListRequest;
import org.apache.hadoop.ozone.protocol.proto
.KeySpaceManagerProtocolProtos.ServiceListResponse;
import org.apache.hadoop.ozone.protocol.proto.OzoneProtos;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.IOException;
import java.util.List;
import java.util.stream.Collectors;
/**
* This class is the server-side translator that forwards requests received on
@ -501,4 +509,19 @@ public AllocateBlockResponse allocateBlock(RpcController controller,
}
return resp.build();
}
@Override
public ServiceListResponse getServiceList(RpcController controller,
ServiceListRequest request) throws ServiceException {
ServiceListResponse.Builder resp = ServiceListResponse.newBuilder();
try {
resp.addAllServiceInfo(impl.getServiceList().stream()
.map(ServiceInfo::getProtobuf)
.collect(Collectors.toList()));
resp.setStatus(Status.OK);
} catch (IOException e) {
resp.setStatus(exceptionToResponseStatus(e));
}
return resp.build();
}
}

View File

@ -21,6 +21,7 @@
import org.apache.hadoop.fs.StorageType;
import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.hdfs.server.datanode.ObjectStoreHandler;
import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.ozone.MiniOzoneClassicCluster;
import org.apache.hadoop.ozone.MiniOzoneCluster;
import org.apache.hadoop.ozone.OzoneConfigKeys;
@ -30,6 +31,10 @@
import org.apache.hadoop.ozone.client.rest.OzoneException;
import org.apache.hadoop.ozone.ksm.exceptions.KSMException;
import org.apache.hadoop.ozone.scm.SCMStorage;
import org.apache.hadoop.ozone.ksm.helpers.ServiceInfo;
import org.apache.hadoop.ozone.protocol.proto
.KeySpaceManagerProtocolProtos.ServicePort;
import org.apache.hadoop.ozone.protocol.proto.OzoneProtos;
import org.apache.hadoop.ozone.web.handlers.BucketArgs;
import org.apache.hadoop.ozone.web.handlers.KeyArgs;
import org.apache.hadoop.ozone.web.handlers.UserArgs;
@ -66,6 +71,7 @@
import java.io.OutputStream;
import java.nio.file.Path;
import java.nio.file.Paths;
import java.net.InetSocketAddress;
import java.text.ParseException;
import java.util.HashSet;
import java.util.LinkedList;
@ -80,6 +86,9 @@
import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_OPEN_KEY_CLEANUP_SERVICE_INTERVAL_SECONDS;
import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_OPEN_KEY_EXPIRE_THRESHOLD_SECONDS;
import static org.apache.hadoop.ozone.OzoneConsts.DELETING_KEY_PREFIX;
import static org.apache.hadoop.ozone.ksm.KSMConfigKeys.OZONE_KSM_ADDRESS_KEY;
import static org.apache.hadoop.scm.ScmConfigKeys.OZONE_SCM_CLIENT_ADDRESS_KEY;
/**
* Test Key Space Manager operation in distributed handler scenario.
*/
@ -1063,7 +1072,7 @@ public void testWriteSize() throws IOException, OzoneException {
*/
@Test
public void testGetScmInfo() throws IOException {
ScmInfo info = cluster.getKeySpaceManager().getScmInfo(conf);
ScmInfo info = cluster.getKeySpaceManager().getScmInfo();
Assert.assertEquals(clusterId, info.getClusterId());
Assert.assertEquals(scmId, info.getScmId());
}
@ -1185,4 +1194,29 @@ public void testKSMInitializationFailure() throws Exception {
exception.expectMessage("SCM version info mismatch.");
KeySpaceManager.createKSM(null, conf);
}
@Test
public void testGetServiceList() throws IOException {
long numGetServiceListCalls = ksmMetrics.getNumGetServiceLists();
List<ServiceInfo> services = cluster.getKeySpaceManager().getServiceList();
Assert.assertEquals(numGetServiceListCalls + 1,
ksmMetrics.getNumGetServiceLists());
ServiceInfo ksmInfo = services.stream().filter(
a -> a.getNodeType().equals(OzoneProtos.NodeType.KSM))
.collect(Collectors.toList()).get(0);
InetSocketAddress ksmAddress = new InetSocketAddress(ksmInfo.getHostname(),
ksmInfo.getPort(ServicePort.Type.RPC));
Assert.assertEquals(NetUtils.createSocketAddr(
conf.get(OZONE_KSM_ADDRESS_KEY)), ksmAddress);
ServiceInfo scmInfo = services.stream().filter(
a -> a.getNodeType().equals(OzoneProtos.NodeType.SCM))
.collect(Collectors.toList()).get(0);
InetSocketAddress scmAddress = new InetSocketAddress(scmInfo.getHostname(),
scmInfo.getPort(ServicePort.Type.RPC));
Assert.assertEquals(NetUtils.createSocketAddr(
conf.get(OZONE_SCM_CLIENT_ADDRESS_KEY)), scmAddress);
}
}