|
@@ -0,0 +1,176 @@
|
|
|
|
+/**
|
|
|
|
+ * 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.
|
|
|
|
+ */
|
|
|
|
+
|
|
|
|
+/**
|
|
|
|
+ * These .proto interfaces are private and unstable.
|
|
|
|
+ * Please see http://wiki.apache.org/hadoop/Compatibility
|
|
|
|
+ * for what changes are allowed for a *unstable* .proto interface.
|
|
|
|
+ */
|
|
|
|
+
|
|
|
|
+option java_package = "org.apache.hadoop.ozone.protocol.proto";
|
|
|
|
+
|
|
|
|
+option java_outer_classname = "StorageContainerDatanodeProtocolProtos";
|
|
|
|
+
|
|
|
|
+option java_generic_services = true;
|
|
|
|
+
|
|
|
|
+option java_generate_equals_and_hash = true;
|
|
|
|
+
|
|
|
|
+package hadoop.hdfs;
|
|
|
|
+import "hdfs.proto";
|
|
|
|
+import "HdfsServer.proto";
|
|
|
|
+import "DatanodeProtocol.proto";
|
|
|
|
+import "DatanodeContainerProtocol.proto";
|
|
|
|
+
|
|
|
|
+
|
|
|
|
+/**
|
|
|
|
+* This message is send by data node to indicate that it is alive or it is
|
|
|
|
+* registering with the node manager.
|
|
|
|
+*/
|
|
|
|
+message SCMHeartbeatRequestProto {
|
|
|
|
+ required DatanodeIDProto datanodeID = 1;
|
|
|
|
+}
|
|
|
|
+
|
|
|
|
+
|
|
|
|
+/**
|
|
|
|
+ * Request for version info of the software stack on the server.
|
|
|
|
+ */
|
|
|
|
+message SCMVersionRequestProto {
|
|
|
|
+
|
|
|
|
+}
|
|
|
|
+
|
|
|
|
+/**
|
|
|
|
+* Generic response that is send to a version request. This allows keys to be
|
|
|
|
+* added on the fly and protocol to remain stable.
|
|
|
|
+*/
|
|
|
|
+message SCMVersionResponseProto {
|
|
|
|
+ required uint32 softwareVersion = 1;
|
|
|
|
+ repeated hadoop.hdfs.ozone.KeyValue keys = 2;
|
|
|
|
+}
|
|
|
|
+
|
|
|
|
+/**
|
|
|
|
+ * Datanode ID returned by the SCM. This is similar to name node
|
|
|
|
+ * registeration of a datanode.
|
|
|
|
+ */
|
|
|
|
+message RegisteredCmdResponseProto {
|
|
|
|
+ required string datanodeUUID = 1;
|
|
|
|
+ required string clusterID = 2;
|
|
|
|
+}
|
|
|
|
+
|
|
|
|
+/*
|
|
|
|
+ * These are commands returned by SCM for to the datanode to execute.
|
|
|
|
+ */
|
|
|
|
+
|
|
|
|
+message SCMHeartbeatResponseProto {
|
|
|
|
+ enum Type {
|
|
|
|
+ nullCmd = 1;
|
|
|
|
+ registeredCmd = 2; // Returns the datanode ID after registeration.
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ required Type cmdType = 1; // Type of the command
|
|
|
|
+ optional RegisteredCmdResponseProto registerNode = 2;
|
|
|
|
+
|
|
|
|
+}
|
|
|
|
+
|
|
|
|
+
|
|
|
|
+
|
|
|
|
+/**
|
|
|
|
+ * Protocol used from a datanode to StorageContainerManager.
|
|
|
|
+ *
|
|
|
|
+ * Please see the request and response messages for details of the RPC calls.
|
|
|
|
+ *
|
|
|
|
+ * Here is a simple state diagram that shows how a datanode would boot up and
|
|
|
|
+ * communicate with SCM.
|
|
|
|
+ *
|
|
|
|
+ * +-----------------------+
|
|
|
|
+ * | Start |
|
|
|
|
+ * +----------+------------+
|
|
|
|
+ * |
|
|
|
|
+ * |
|
|
|
|
+ * |
|
|
|
|
+ * |
|
|
|
|
+ * |
|
|
|
|
+ * |
|
|
|
|
+ * |
|
|
|
|
+ * +----------v-------------+
|
|
|
|
+ * | Searching for SCM +------------+
|
|
|
|
+ * +----------+-------------+ |
|
|
|
|
+ * | |
|
|
|
|
+ * | |
|
|
|
|
+ * | +----------v-------------+
|
|
|
|
+ * | | Register if needed |
|
|
|
|
+ * | +-----------+------------+
|
|
|
|
+ * | |
|
|
|
|
+ * v |
|
|
|
|
+ * +-----------+----------------+ |
|
|
|
|
+ * +---------+ Heartbeat state <--------+
|
|
|
|
+ * | +--------^-------------------+
|
|
|
|
+ * | |
|
|
|
|
+ * | |
|
|
|
|
+ * | |
|
|
|
|
+ * | |
|
|
|
|
+ * | |
|
|
|
|
+ * | |
|
|
|
|
+ * | |
|
|
|
|
+ * +------------------+
|
|
|
|
+ *
|
|
|
|
+ *
|
|
|
|
+ *
|
|
|
|
+ * Here is how this protocol is used by the datanode. When a datanode boots up
|
|
|
|
+ * it moves into a stated called SEARCHING_SCM. In this state datanode is
|
|
|
|
+ * trying to establish communication with the SCM. The address of the SCMs are
|
|
|
|
+ * retrieved from the configuration information.
|
|
|
|
+ *
|
|
|
|
+ * In the SEARCHING_SCM state, only rpc call made by datanode is a getVersion
|
|
|
|
+ * call to SCM. Once any of the SCMs reply, datanode checks if it has a local
|
|
|
|
+ * persisted datanode ID. If it has this means that this datanode is already
|
|
|
|
+ * registered with some SCM. If this file is not found, datanode assumes that
|
|
|
|
+ * it needs to do a registration.
|
|
|
|
+ *
|
|
|
|
+ * If registration is need datanode moves into REGISTERING_NODE state. It will
|
|
|
|
+ * send a register call with datanodeID data structure, but with datanode UUID
|
|
|
|
+ * will be set to an empty string.
|
|
|
|
+ *
|
|
|
|
+ * The response to the command contains the datanode UUID and clusterID. This
|
|
|
|
+ * information is persisted by the datanode and moves into heartbeat state.
|
|
|
|
+ *
|
|
|
|
+ * Once in the heartbeat state, datanode sends heartbeats and container reports
|
|
|
|
+ * to SCM and process commands issued by SCM until it is shutdown.
|
|
|
|
+ *
|
|
|
|
+ * For time being we are going to use SCMHeartbeatResponseProto as the return
|
|
|
|
+ * type for register and sendheartbeat messages.
|
|
|
|
+ */
|
|
|
|
+service StorageContainerDatanodeProtocolService {
|
|
|
|
+
|
|
|
|
+ /**
|
|
|
|
+ * Gets the version information from the SCM.
|
|
|
|
+ */
|
|
|
|
+ rpc getVersion(SCMVersionRequestProto) returns (SCMVersionResponseProto);
|
|
|
|
+
|
|
|
|
+ /**
|
|
|
|
+ * Registers a data node with SCM.
|
|
|
|
+ */
|
|
|
|
+ rpc register(SCMHeartbeatRequestProto) returns (SCMHeartbeatResponseProto);
|
|
|
|
+
|
|
|
|
+ /**
|
|
|
|
+ * Send heartbeat from datanode to SCM. HB's under SCM looks more
|
|
|
|
+ * like life line protocol than HB's under HDFS. In other words, it is
|
|
|
|
+ * extremely light weight and contains no data payload.
|
|
|
|
+ */
|
|
|
|
+ rpc sendHeartbeat(SCMHeartbeatRequestProto) returns (SCMHeartbeatResponseProto);
|
|
|
|
+
|
|
|
|
+}
|