|
@@ -22,6 +22,7 @@ import com.google.common.base.Preconditions;
|
|
|
import com.google.protobuf.BlockingService;
|
|
|
import org.apache.hadoop.classification.InterfaceAudience;
|
|
|
import org.apache.hadoop.hdfs.DFSUtil;
|
|
|
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos;
|
|
|
import org.apache.hadoop.ipc.Client;
|
|
|
import org.apache.hadoop.ipc.ProtobufRpcEngine;
|
|
|
import org.apache.hadoop.ipc.RPC;
|
|
@@ -53,8 +54,13 @@ import org.apache.hadoop.ozone.protocolPB
|
|
|
.KeySpaceManagerProtocolServerSideTranslatorPB;
|
|
|
import org.apache.hadoop.scm.ScmInfo;
|
|
|
import org.apache.hadoop.scm.protocol.ScmBlockLocationProtocol;
|
|
|
-import org.apache.hadoop.scm.protocolPB.ScmBlockLocationProtocolClientSideTranslatorPB;
|
|
|
+import org.apache.hadoop.scm.protocol.StorageContainerLocationProtocol;
|
|
|
+import org.apache.hadoop.scm.protocolPB
|
|
|
+ .ScmBlockLocationProtocolClientSideTranslatorPB;
|
|
|
import org.apache.hadoop.scm.protocolPB.ScmBlockLocationProtocolPB;
|
|
|
+import org.apache.hadoop.scm.protocolPB
|
|
|
+ .StorageContainerLocationProtocolClientSideTranslatorPB;
|
|
|
+import org.apache.hadoop.scm.protocolPB.StorageContainerLocationProtocolPB;
|
|
|
import org.apache.hadoop.security.UserGroupInformation;
|
|
|
import org.apache.hadoop.util.GenericOptionsParser;
|
|
|
import org.apache.hadoop.util.StringUtils;
|
|
@@ -66,6 +72,7 @@ import java.io.IOException;
|
|
|
import java.io.PrintStream;
|
|
|
import java.net.InetSocketAddress;
|
|
|
import java.util.ArrayList;
|
|
|
+import java.util.EnumSet;
|
|
|
import java.util.HashMap;
|
|
|
import java.util.List;
|
|
|
import java.util.Map;
|
|
@@ -80,6 +87,8 @@ import static org.apache.hadoop.ozone.ksm.KSMConfigKeys
|
|
|
import static org.apache.hadoop.ozone.protocol.proto
|
|
|
.KeySpaceManagerProtocolProtos.KeySpaceManagerService
|
|
|
.newReflectiveBlockingService;
|
|
|
+import static org.apache.hadoop.ozone.protocol.proto.OzoneProtos
|
|
|
+ .NodeState.HEALTHY;
|
|
|
import static org.apache.hadoop.util.ExitUtil.terminate;
|
|
|
|
|
|
/**
|
|
@@ -133,6 +142,7 @@ public class KeySpaceManager extends ServiceRuntimeInfoImpl
|
|
|
private final KeySpaceManagerHttpServer httpServer;
|
|
|
private final KSMStorage ksmStorage;
|
|
|
private final ScmBlockLocationProtocol scmBlockClient;
|
|
|
+ private final StorageContainerLocationProtocol scmContainerClient;
|
|
|
private ObjectName ksmInfoBeanName;
|
|
|
|
|
|
private KeySpaceManager(OzoneConfiguration conf) throws IOException {
|
|
@@ -140,6 +150,7 @@ public class KeySpaceManager extends ServiceRuntimeInfoImpl
|
|
|
configuration = conf;
|
|
|
ksmStorage = new KSMStorage(conf);
|
|
|
scmBlockClient = getScmBlockClient(configuration);
|
|
|
+ scmContainerClient = getScmContainerClient(configuration);
|
|
|
if (ksmStorage.getState() != StorageState.INITIALIZED) {
|
|
|
throw new KSMException("KSM not initialized.",
|
|
|
ResultCodes.KSM_NOT_INITIALIZED);
|
|
@@ -199,6 +210,29 @@ public class KeySpaceManager extends ServiceRuntimeInfoImpl
|
|
|
return scmBlockLocationClient;
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * Returns a scm container client.
|
|
|
+ *
|
|
|
+ * @return {@link StorageContainerLocationProtocol}
|
|
|
+ * @throws IOException
|
|
|
+ */
|
|
|
+ private static StorageContainerLocationProtocol getScmContainerClient(
|
|
|
+ OzoneConfiguration conf) throws IOException {
|
|
|
+ RPC.setProtocolEngine(conf, StorageContainerLocationProtocolPB.class,
|
|
|
+ ProtobufRpcEngine.class);
|
|
|
+ long scmVersion =
|
|
|
+ RPC.getProtocolVersion(StorageContainerLocationProtocolPB.class);
|
|
|
+ InetSocketAddress scmAddr = OzoneClientUtils.getScmAddressForClients(
|
|
|
+ conf);
|
|
|
+ StorageContainerLocationProtocolClientSideTranslatorPB scmContainerClient =
|
|
|
+ new StorageContainerLocationProtocolClientSideTranslatorPB(
|
|
|
+ RPC.getProxy(StorageContainerLocationProtocolPB.class, scmVersion,
|
|
|
+ scmAddr, UserGroupInformation.getCurrentUser(), conf,
|
|
|
+ NetUtils.getDefaultSocketFactory(conf),
|
|
|
+ Client.getRpcTimeout(conf)));
|
|
|
+ return scmContainerClient;
|
|
|
+ }
|
|
|
+
|
|
|
@VisibleForTesting
|
|
|
public ScmInfo getScmInfo() throws IOException {
|
|
|
return scmBlockClient.getScmInfo();
|
|
@@ -813,6 +847,7 @@ public class KeySpaceManager extends ServiceRuntimeInfoImpl
|
|
|
.setValue(httpServer.getHttpsAddress().getPort())
|
|
|
.build());
|
|
|
}
|
|
|
+ services.add(ksmServiceInfoBuilder.build());
|
|
|
|
|
|
// For client we have to return SCM with container protocol port,
|
|
|
// not block protocol.
|
|
@@ -824,11 +859,33 @@ public class KeySpaceManager extends ServiceRuntimeInfoImpl
|
|
|
.addServicePort(ServicePort.newBuilder()
|
|
|
.setType(ServicePort.Type.RPC)
|
|
|
.setValue(scmAddr.getPort()).build());
|
|
|
+ services.add(scmServiceInfoBuilder.build());
|
|
|
|
|
|
- // TODO: REST servers (datanode) details to be added later.
|
|
|
+ List<OzoneProtos.Node> nodes = scmContainerClient.queryNode(
|
|
|
+ EnumSet.of(HEALTHY), OzoneProtos.QueryScope.CLUSTER, "")
|
|
|
+ .getNodesList();
|
|
|
+
|
|
|
+ for (OzoneProtos.Node node : nodes) {
|
|
|
+ HdfsProtos.DatanodeIDProto datanode = node.getNodeID();
|
|
|
+
|
|
|
+ ServiceInfo.Builder dnServiceInfoBuilder = ServiceInfo.newBuilder()
|
|
|
+ .setNodeType(OzoneProtos.NodeType.DATANODE)
|
|
|
+ .setHostname(datanode.getHostName());
|
|
|
+
|
|
|
+ dnServiceInfoBuilder.addServicePort(ServicePort.newBuilder()
|
|
|
+ .setType(ServicePort.Type.HTTP)
|
|
|
+ .setValue(datanode.getInfoPort())
|
|
|
+ .build());
|
|
|
+
|
|
|
+ if (datanode.hasInfoSecurePort() && datanode.getInfoSecurePort() > 0) {
|
|
|
+ dnServiceInfoBuilder.addServicePort(ServicePort.newBuilder()
|
|
|
+ .setType(ServicePort.Type.HTTPS)
|
|
|
+ .setValue(datanode.getInfoSecurePort())
|
|
|
+ .build());
|
|
|
+ }
|
|
|
+ services.add(dnServiceInfoBuilder.build());
|
|
|
+ }
|
|
|
|
|
|
- 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
|