|
@@ -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.KsmKeyInfo;
|
|
|
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.metrics2.util.MBeans;
|
|
|
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 javax.management.ObjectName;
|
|
|
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 class KeySpaceManager extends ServiceRuntimeInfoImpl
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ private final OzoneConfiguration configuration;
|
|
|
private final RPC.Server ksmRpcServer;
|
|
|
private final InetSocketAddress ksmRpcAddress;
|
|
|
private final KSMMetadataManager metadataManager;
|
|
@@ -125,11 +132,14 @@ public class KeySpaceManager extends ServiceRuntimeInfoImpl
|
|
|
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 @@ public class KeySpaceManager extends ServiceRuntimeInfoImpl
|
|
|
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 @@ public class KeySpaceManager extends ServiceRuntimeInfoImpl
|
|
|
}
|
|
|
|
|
|
@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 @@ public class KeySpaceManager extends ServiceRuntimeInfoImpl
|
|
|
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;
|
|
|
+ }
|
|
|
}
|