You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@ratis.apache.org by GitBox <gi...@apache.org> on 2020/10/26 01:43:38 UTC

[GitHub] [incubator-ratis] runzhiwang opened a new pull request #232: RATIS-1106. Add type for DataStream

runzhiwang opened a new pull request #232:
URL: https://github.com/apache/incubator-ratis/pull/232


   ## What changes were proposed in this pull request?
   
   Add DataStreamRequestType: HEAD, DATA, CLOSE, START_TRANSACTION
   
   ## What is the link to the Apache JIRA
   
   https://issues.apache.org/jira/browse/RATIS-1106
   
   ## How was this patch tested?
   
   New assert


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-ratis] runzhiwang commented on a change in pull request #232: RATIS-1106. Add type for DataStream

Posted by GitBox <gi...@apache.org>.
runzhiwang commented on a change in pull request #232:
URL: https://github.com/apache/incubator-ratis/pull/232#discussion_r512392787



##########
File path: ratis-common/src/main/java/org/apache/ratis/protocol/DataStreamPacketHeader.java
##########
@@ -19,29 +19,17 @@
 package org.apache.ratis.protocol;
 
 import org.apache.ratis.datastream.impl.DataStreamPacketImpl;
+import org.apache.ratis.proto.RaftProtos.DataStreamPacketHeaderProto.Type;
 import org.apache.ratis.util.SizeInBytes;
 
-import java.util.function.LongSupplier;
-
 /** The header format is streamId, streamOffset, dataLength. */
 public class DataStreamPacketHeader extends DataStreamPacketImpl {
-  private static final SizeInBytes SIZE = SizeInBytes.valueOf(24);
-
-  public static int getSize() {
-    return SIZE.getSizeInt();
-  }
-
-  public static DataStreamPacketHeader read(LongSupplier readLong, int readableBytes) {
-    if (readableBytes < getSize()) {
-      return null;
-    }
-    return new DataStreamPacketHeader(readLong.getAsLong(), readLong.getAsLong(), readLong.getAsLong());
-  }
+  private static final SizeInBytes SIZE_OF_HEADER_LEN = SizeInBytes.valueOf(4);

Review comment:
       It's not header size, it's size of header size. For example, if header size is 24, we only need an int to store 24, and the size of int is 4.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-ratis] runzhiwang commented on pull request #232: RATIS-1106. Add type for DataStream

Posted by GitBox <gi...@apache.org>.
runzhiwang commented on pull request #232:
URL: https://github.com/apache/incubator-ratis/pull/232#issuecomment-716326514


   @szetszwo Thanks for review. I will use Protobuf in this PR.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-ratis] amaliujia commented on pull request #232: RATIS-1106. Add type for DataStream

Posted by GitBox <gi...@apache.org>.
amaliujia commented on pull request #232:
URL: https://github.com/apache/incubator-ratis/pull/232#issuecomment-716978629


   +1


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-ratis] amaliujia commented on a change in pull request #232: RATIS-1106. Add type for DataStream

Posted by GitBox <gi...@apache.org>.
amaliujia commented on a change in pull request #232:
URL: https://github.com/apache/incubator-ratis/pull/232#discussion_r512393398



##########
File path: ratis-common/src/main/java/org/apache/ratis/datastream/impl/DataStreamReplyByteBuffer.java
##########
@@ -32,15 +33,16 @@
   private final boolean success;
 
   public DataStreamReplyByteBuffer(long streamId, long streamOffset, ByteBuffer buffer,
-      long bytesWritten, boolean success ) {
-    super(streamId, streamOffset, buffer);
+      long bytesWritten, boolean success, Type type) {
+    super(streamId, streamOffset, buffer, type);
 
     this.success = success;
     this.bytesWritten = bytesWritten;
   }
 
   public DataStreamReplyByteBuffer(DataStreamReplyHeader header, ByteBuffer buffer) {
-    this(header.getStreamId(), header.getStreamOffset(), buffer, header.getBytesWritten(), header.isSuccess());
+    this(header.getStreamId(), header.getStreamOffset(), buffer, header.getBytesWritten(), header.isSuccess(),
+        header.getType());

Review comment:
       is there a need to include type into replies?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-ratis] runzhiwang commented on a change in pull request #232: RATIS-1106. Add type for DataStream

Posted by GitBox <gi...@apache.org>.
runzhiwang commented on a change in pull request #232:
URL: https://github.com/apache/incubator-ratis/pull/232#discussion_r512393161



##########
File path: ratis-proto/src/main/proto/Raft.proto
##########
@@ -279,6 +279,30 @@ message RaftClientRequestProto {
   }
 }
 
+message DataStreamPacketHeaderProto {
+  enum Type {
+    STREAM_HEADER = 0;
+    STREAM_DATA = 1;
+    STREAM_CLOSE = 2;
+    START_TRANSACTION = 3;
+  }
+
+  uint64 streamId = 1;
+  uint64 streamOffset = 2;
+  Type type = 3;
+  uint64 dataLength = 4;
+}
+
+message DataStreamRequestHeaderProto {
+  DataStreamPacketHeaderProto packetHeader = 1;
+}
+
+message DataStreamReplyHeaderProto {
+  DataStreamPacketHeaderProto packetHeader = 1;
+  uint64 bytesWritten = 2;
+  bool success = 3;

Review comment:
       isSuccess is the name of method.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-ratis] amaliujia commented on a change in pull request #232: RATIS-1106. Add type for DataStream

Posted by GitBox <gi...@apache.org>.
amaliujia commented on a change in pull request #232:
URL: https://github.com/apache/incubator-ratis/pull/232#discussion_r512391838



##########
File path: ratis-common/src/main/java/org/apache/ratis/protocol/DataStreamPacketHeader.java
##########
@@ -19,29 +19,17 @@
 package org.apache.ratis.protocol;
 
 import org.apache.ratis.datastream.impl.DataStreamPacketImpl;
+import org.apache.ratis.proto.RaftProtos.DataStreamPacketHeaderProto.Type;
 import org.apache.ratis.util.SizeInBytes;
 
-import java.util.function.LongSupplier;
-
 /** The header format is streamId, streamOffset, dataLength. */
 public class DataStreamPacketHeader extends DataStreamPacketImpl {
-  private static final SizeInBytes SIZE = SizeInBytes.valueOf(24);
-
-  public static int getSize() {
-    return SIZE.getSizeInt();
-  }
-
-  public static DataStreamPacketHeader read(LongSupplier readLong, int readableBytes) {
-    if (readableBytes < getSize()) {
-      return null;
-    }
-    return new DataStreamPacketHeader(readLong.getAsLong(), readLong.getAsLong(), readLong.getAsLong());
-  }
+  private static final SizeInBytes SIZE_OF_HEADER_LEN = SizeInBytes.valueOf(4);

Review comment:
       why header size is 4 now?

##########
File path: ratis-proto/src/main/proto/Raft.proto
##########
@@ -279,6 +279,30 @@ message RaftClientRequestProto {
   }
 }
 
+message DataStreamPacketHeaderProto {
+  enum Type {
+    STREAM_HEADER = 0;
+    STREAM_DATA = 1;
+    STREAM_CLOSE = 2;
+    START_TRANSACTION = 3;
+  }
+
+  uint64 streamId = 1;
+  uint64 streamOffset = 2;
+  Type type = 3;
+  uint64 dataLength = 4;
+}
+
+message DataStreamRequestHeaderProto {
+  DataStreamPacketHeaderProto packetHeader = 1;
+}
+
+message DataStreamReplyHeaderProto {
+  DataStreamPacketHeaderProto packetHeader = 1;
+  uint64 bytesWritten = 2;
+  bool success = 3;

Review comment:
       `isSuccess`?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-ratis] runzhiwang commented on pull request #232: RATIS-1106. Add type for DataStream

Posted by GitBox <gi...@apache.org>.
runzhiwang commented on pull request #232:
URL: https://github.com/apache/incubator-ratis/pull/232#issuecomment-716258085


   @szetszwo Could you help review this ? Thanks a lot.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-ratis] runzhiwang commented on pull request #232: RATIS-1106. Add type for DataStream

Posted by GitBox <gi...@apache.org>.
runzhiwang commented on pull request #232:
URL: https://github.com/apache/incubator-ratis/pull/232#issuecomment-717024413


   @szetszwo Thanks for merging it. @amaliujia Thanks for review.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-ratis] szetszwo merged pull request #232: RATIS-1106. Add type for DataStream

Posted by GitBox <gi...@apache.org>.
szetszwo merged pull request #232:
URL: https://github.com/apache/incubator-ratis/pull/232


   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-ratis] runzhiwang commented on pull request #232: RATIS-1106. Add type for DataStream

Posted by GitBox <gi...@apache.org>.
runzhiwang commented on pull request #232:
URL: https://github.com/apache/incubator-ratis/pull/232#issuecomment-716923749


   @szetszwo I have updated the patch. Could you help review it again ?


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-ratis] szetszwo commented on a change in pull request #232: RATIS-1106. Add type for DataStream

Posted by GitBox <gi...@apache.org>.
szetszwo commented on a change in pull request #232:
URL: https://github.com/apache/incubator-ratis/pull/232#discussion_r511726875



##########
File path: ratis-common/src/main/java/org/apache/ratis/protocol/DataStreamRequestType.java
##########
@@ -0,0 +1,25 @@
+/*
+ * 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.
+ */
+package org.apache.ratis.protocol;
+
+public enum DataStreamRequestType {

Review comment:
       This is indeed a packet type.  Let's change this as an inner class DataStreamPacket.Type.  

##########
File path: ratis-common/src/main/java/org/apache/ratis/protocol/DataStreamRequestType.java
##########
@@ -0,0 +1,25 @@
+/*
+ * 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.
+ */
+package org.apache.ratis.protocol;
+
+public enum DataStreamRequestType {
+  HEAD,
+  DATA,
+  CLOSE,

Review comment:
       Let's call them STREAM_HEADER, STREAM_DATA and STREAM_CLOSE.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-ratis] amaliujia commented on a change in pull request #232: RATIS-1106. Add type for DataStream

Posted by GitBox <gi...@apache.org>.
amaliujia commented on a change in pull request #232:
URL: https://github.com/apache/incubator-ratis/pull/232#discussion_r512414662



##########
File path: ratis-common/src/main/java/org/apache/ratis/protocol/DataStreamPacketHeader.java
##########
@@ -19,29 +19,17 @@
 package org.apache.ratis.protocol;
 
 import org.apache.ratis.datastream.impl.DataStreamPacketImpl;
+import org.apache.ratis.proto.RaftProtos.DataStreamPacketHeaderProto.Type;
 import org.apache.ratis.util.SizeInBytes;
 
-import java.util.function.LongSupplier;
-
 /** The header format is streamId, streamOffset, dataLength. */
 public class DataStreamPacketHeader extends DataStreamPacketImpl {
-  private static final SizeInBytes SIZE = SizeInBytes.valueOf(24);
-
-  public static int getSize() {
-    return SIZE.getSizeInt();
-  }
-
-  public static DataStreamPacketHeader read(LongSupplier readLong, int readableBytes) {
-    if (readableBytes < getSize()) {
-      return null;
-    }
-    return new DataStreamPacketHeader(readLong.getAsLong(), readLong.getAsLong(), readLong.getAsLong());
-  }
+  private static final SizeInBytes SIZE_OF_HEADER_LEN = SizeInBytes.valueOf(4);

Review comment:
       thanks for the clarification! 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org