|
@@ -0,0 +1,314 @@
|
|
|
+/**
|
|
|
+ * 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://hadoop.apache.org/docs/stable/hadoop-project-dist/hadoop-common/InterfaceClassification.html
|
|
|
+ * for what changes are allowed for a *Unstable* .proto interface.
|
|
|
+ */
|
|
|
+
|
|
|
+// This file contains protocol buffers that are used to transfer data
|
|
|
+// to and from the datanode.
|
|
|
+option java_package = "org.apache.hadoop.hdfs.ozone.protocol.proto";
|
|
|
+option java_outer_classname = "ContainerProtos";
|
|
|
+option java_generate_equals_and_hash = true;
|
|
|
+package hadoop.hdfs.ozone;
|
|
|
+import "hdfs.proto";
|
|
|
+
|
|
|
+/**
|
|
|
+ * Commands that are used to manipulate the state of containers on a datanode.
|
|
|
+ *
|
|
|
+ * These commands allow us to work against the datanode - from
|
|
|
+ * StorageContainer Manager as well as clients.
|
|
|
+ *
|
|
|
+ * 1. CreateContainer - This call is usually made by Storage Container
|
|
|
+ * manager, when we need to create a new container on a given datanode.
|
|
|
+ *
|
|
|
+ * 2. ReadContainer - Allows end user to stat a container. For example
|
|
|
+ * this allows us to return the metadata of a container.
|
|
|
+ *
|
|
|
+ * 3. UpdateContainer - Updates a container metadata.
|
|
|
+
|
|
|
+ * 4. DeleteContainer - This call is made to delete a container.
|
|
|
+ *
|
|
|
+ * 5. ListContainer - Returns the list of containers on this
|
|
|
+ * datanode. This will be used by tests and tools.
|
|
|
+ *
|
|
|
+ * 6. CreateKey - Given a valid container, creates a key.
|
|
|
+ *
|
|
|
+ * 7. ReadKey - Allows user to read the metadata of a Key.
|
|
|
+ *
|
|
|
+ * 8. UpdateKey - Updates the metadata of a Key.
|
|
|
+ *
|
|
|
+ * 9. DeleteKey - Deletes a given key.
|
|
|
+ *
|
|
|
+ * 10. ListKey - Returns a list of keys that are present inside
|
|
|
+ * a given container.
|
|
|
+ *
|
|
|
+ * 11. ReadChunk - Allows us to read a chunk.
|
|
|
+ *
|
|
|
+ * 12. DeleteChunk - Delete an unused chunk.
|
|
|
+ *
|
|
|
+ * 13. WriteChunk - Allows us to write a chunk
|
|
|
+ *
|
|
|
+ * 14. ListChunk - Given a Container/Key returns the list of Chunks.
|
|
|
+ *
|
|
|
+ */
|
|
|
+
|
|
|
+enum Type {
|
|
|
+ CreateContainer = 1;
|
|
|
+ ReadContainer = 2;
|
|
|
+ UpdateContainer = 3;
|
|
|
+ DeleteContainer = 4;
|
|
|
+ ListContainer = 5;
|
|
|
+
|
|
|
+ CreateKey = 6;
|
|
|
+ Readkey = 7;
|
|
|
+ UpdateKey = 8;
|
|
|
+ DeleteKey = 9;
|
|
|
+ ListKey = 10;
|
|
|
+
|
|
|
+ ReadChunk = 11;
|
|
|
+ DeleteChunk = 12;
|
|
|
+ WriteChunk = 13;
|
|
|
+ ListChunk = 14;
|
|
|
+}
|
|
|
+
|
|
|
+
|
|
|
+message ContainerCommandRequestProto {
|
|
|
+ required Type cmdType = 1; // Type of the command
|
|
|
+
|
|
|
+ // A string that identifies this command, we generate Trace ID in Ozone
|
|
|
+ // frontend and this allows us to trace that command all over ozone.
|
|
|
+ optional string traceID = 2;
|
|
|
+
|
|
|
+ // One of the following command is available when the corresponding
|
|
|
+ // cmdType is set. At the protocol level we allow only
|
|
|
+ // one command in each packet.
|
|
|
+ // TODO : Upgrade to Protobuf 2.6 or later.
|
|
|
+ optional CreateContainerRequestProto createContainer = 3;
|
|
|
+ optional ReadContainerRequestProto readContainer = 4;
|
|
|
+ optional UpdateContainerRequestProto updateContainer = 5;
|
|
|
+ optional DeleteContainerRequestProto deleteContainer = 6;
|
|
|
+ optional ListContainerRequestProto listContainer = 7;
|
|
|
+
|
|
|
+ optional CreateKeyRequestProto createKey = 8;
|
|
|
+ optional ReadKeyRequestProto readKey = 9;
|
|
|
+ optional UpdateKeyRequestProto updateKey = 10;
|
|
|
+ optional DeleteKeyRequestProto deleteKey = 11;
|
|
|
+ optional ListKeyRequestProto listKey = 12;
|
|
|
+
|
|
|
+ optional ReadChunkRequestProto readChunk = 13;
|
|
|
+ optional WriteChunkRequestProto writeChunk = 14;
|
|
|
+ optional DeleteChunkRequestProto deleteChunk = 15;
|
|
|
+ optional ListChunkRequestProto listChunk = 16;
|
|
|
+}
|
|
|
+
|
|
|
+message ContainerCommandResponseProto {
|
|
|
+ required Type cmdType = 1;
|
|
|
+ optional string traceID = 2;
|
|
|
+
|
|
|
+ optional CreateContainerResponseProto createContainer = 3;
|
|
|
+ optional ReadContainerResponseProto readContainer = 4;
|
|
|
+ optional UpdateContainerResponseProto updateContainer = 5;
|
|
|
+ optional DeleteContainerResponseProto deleteContainer = 6;
|
|
|
+ optional ListContainerResponseProto listContainer = 7;
|
|
|
+
|
|
|
+ optional CreateKeyResponseProto createKey = 8;
|
|
|
+ optional ReadKeyResponeProto readKey = 9;
|
|
|
+ optional UpdateKeyResponseProto updateKey = 10;
|
|
|
+ optional DeleteKeyResponeProto deleteKey = 11;
|
|
|
+ optional ListKeyResponeProto listKey = 12;
|
|
|
+
|
|
|
+ optional WriteChunkReponseProto writeChunk = 13;
|
|
|
+ optional ReadChunkReponseProto readChunk = 14;
|
|
|
+ optional DeleteChunkResponseProto deleteChunk = 15;
|
|
|
+ optional ListChunkResponseProto listChunk = 16;
|
|
|
+
|
|
|
+}
|
|
|
+
|
|
|
+// A pipeline is composed of one or more datanodes that back a container.
|
|
|
+message Pipeline {
|
|
|
+ required string leaderID = 1;
|
|
|
+ repeated DatanodeIDProto members = 2;
|
|
|
+ optional string containerName = 3;
|
|
|
+}
|
|
|
+
|
|
|
+message KeyValue {
|
|
|
+ required string key = 1;
|
|
|
+ optional string value = 2;
|
|
|
+}
|
|
|
+
|
|
|
+message ContainerData {
|
|
|
+ required string name = 1;
|
|
|
+ repeated KeyValue metadata = 2;
|
|
|
+ optional string containerPath = 3;
|
|
|
+}
|
|
|
+
|
|
|
+// Container Messages.
|
|
|
+message CreateContainerRequestProto {
|
|
|
+ required Pipeline pipeline = 1;
|
|
|
+ required ContainerData containerData = 2;
|
|
|
+}
|
|
|
+
|
|
|
+message CreateContainerResponseProto {
|
|
|
+}
|
|
|
+
|
|
|
+message ReadContainerRequestProto {
|
|
|
+ required Pipeline pipeline = 1;
|
|
|
+ required string name = 2;
|
|
|
+}
|
|
|
+
|
|
|
+message ReadContainerResponseProto {
|
|
|
+ optional ContainerData containerData = 2;
|
|
|
+}
|
|
|
+
|
|
|
+message UpdateContainerRequestProto {
|
|
|
+ required Pipeline pipeline = 1;
|
|
|
+ required ContainerData containerData = 2;
|
|
|
+}
|
|
|
+
|
|
|
+message UpdateContainerResponseProto {
|
|
|
+}
|
|
|
+
|
|
|
+message DeleteContainerRequestProto {
|
|
|
+ required Pipeline pipeline = 1;
|
|
|
+ required string name = 2;
|
|
|
+}
|
|
|
+
|
|
|
+message DeleteContainerResponseProto {
|
|
|
+}
|
|
|
+
|
|
|
+message ListContainerRequestProto {
|
|
|
+ required Pipeline pipeline = 1;
|
|
|
+ optional string prevKey = 2; // if this is not set query from start.
|
|
|
+ required uint32 count = 3; // Max Results to return
|
|
|
+}
|
|
|
+
|
|
|
+message ListContainerResponseProto {
|
|
|
+ repeated ContainerData containerData = 1;
|
|
|
+}
|
|
|
+
|
|
|
+
|
|
|
+message ContainerKeyData {
|
|
|
+ optional string containerName = 1;
|
|
|
+ required string name = 2;
|
|
|
+ repeated KeyValue metadata = 3;
|
|
|
+}
|
|
|
+
|
|
|
+// Key Messages.
|
|
|
+message CreateKeyRequestProto {
|
|
|
+ required Pipeline pipeline = 1;
|
|
|
+ required ContainerKeyData containerKeyData = 2;
|
|
|
+}
|
|
|
+
|
|
|
+message CreateKeyResponseProto {
|
|
|
+}
|
|
|
+
|
|
|
+message ReadKeyRequestProto {
|
|
|
+ required Pipeline pipeline = 1;
|
|
|
+ required ContainerKeyData containerKeyData = 2;
|
|
|
+}
|
|
|
+
|
|
|
+message ReadKeyResponeProto {
|
|
|
+ repeated KeyValue metadata = 1;
|
|
|
+ repeated chunkInfo chunkData = 2;
|
|
|
+}
|
|
|
+
|
|
|
+message UpdateKeyRequestProto {
|
|
|
+ required Pipeline pipeline = 1;
|
|
|
+ required ContainerKeyData containerKeyData = 2;
|
|
|
+}
|
|
|
+
|
|
|
+message UpdateKeyResponseProto {
|
|
|
+}
|
|
|
+
|
|
|
+
|
|
|
+message DeleteKeyRequestProto {
|
|
|
+ required Pipeline pipeline = 1;
|
|
|
+ required string name = 2;
|
|
|
+}
|
|
|
+
|
|
|
+message DeleteKeyResponeProto {
|
|
|
+}
|
|
|
+
|
|
|
+message ListKeyRequestProto {
|
|
|
+ required Pipeline pipeline = 1;
|
|
|
+ required string prevKey = 2;
|
|
|
+ required uint32 count = 3;
|
|
|
+}
|
|
|
+
|
|
|
+message ListKeyResponeProto {
|
|
|
+ repeated ContainerKeyData containerKeyData = 1;
|
|
|
+}
|
|
|
+
|
|
|
+// Chunk Operations
|
|
|
+
|
|
|
+message chunkInfo {
|
|
|
+ required uint64 offset = 1;
|
|
|
+ required uint64 len = 2;
|
|
|
+ optional uint64 checksum = 3;
|
|
|
+ repeated KeyValue metadata = 4;
|
|
|
+}
|
|
|
+
|
|
|
+message WriteChunkRequestProto {
|
|
|
+ required Pipeline pipeline = 1;
|
|
|
+ required string containerName = 2;
|
|
|
+ required string keyName = 3;
|
|
|
+ required chunkInfo chunkData = 4;
|
|
|
+ repeated bytes data = 5;
|
|
|
+}
|
|
|
+
|
|
|
+message WriteChunkReponseProto {
|
|
|
+}
|
|
|
+
|
|
|
+message ReadChunkRequestProto {
|
|
|
+ required Pipeline pipeline = 1;
|
|
|
+ required string containerName = 2;
|
|
|
+ required string keyName = 3;
|
|
|
+ required chunkInfo chunkData = 4;
|
|
|
+}
|
|
|
+
|
|
|
+message ReadChunkReponseProto {
|
|
|
+ required Pipeline pipeline = 1;
|
|
|
+ required chunkInfo chunkData = 2;
|
|
|
+ repeated bytes data = 3;
|
|
|
+}
|
|
|
+
|
|
|
+message DeleteChunkRequestProto {
|
|
|
+ required Pipeline pipeline = 1;
|
|
|
+ required string containerName = 2;
|
|
|
+ required string keyName = 3;
|
|
|
+ required chunkInfo chunkData = 4;
|
|
|
+}
|
|
|
+
|
|
|
+message DeleteChunkResponseProto {
|
|
|
+}
|
|
|
+
|
|
|
+message ListChunkRequestProto {
|
|
|
+ required Pipeline pipeline = 1;
|
|
|
+ required string containerName = 2;
|
|
|
+ required string keyName = 3;
|
|
|
+ required uint64 startOffset = 4;
|
|
|
+ required uint32 count = 5;
|
|
|
+}
|
|
|
+
|
|
|
+message ListChunkResponseProto {
|
|
|
+ repeated chunkInfo chunkData = 1;
|
|
|
+}
|
|
|
+
|