You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@ratis.apache.org by GitBox <gi...@apache.org> on 2022/08/16 16:19:45 UTC

[GitHub] [ratis] swamirishi opened a new pull request, #716: RATIS-1647. Fix EI_EXPOSE_REP in DataStreamRequest implementations

swamirishi opened a new pull request, #716:
URL: https://github.com/apache/ratis/pull/716

   ## What changes were proposed in this pull request?
   The options field is currently an array which would caus [EI_EXPOSE_REP](https://spotbugs.readthedocs.io/en/stable/bugDescriptions.html#ei-may-expose-internal-representation-by-returning-reference-to-mutable-object-ei-expose-rep).
   
   ## What is the link to the Apache JIRA
   https://issues.apache.org/jira/browse/RATIS-1647
   
   ## How was this patch tested?
   Unit Tests
   


-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@ratis.apache.org

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


[GitHub] [ratis] szetszwo commented on a diff in pull request #716: RATIS-1647. Fix EI_EXPOSE_REP in DataStreamRequest implementations

Posted by GitBox <gi...@apache.org>.
szetszwo commented on code in PR #716:
URL: https://github.com/apache/ratis/pull/716#discussion_r977108228


##########
ratis-client/src/main/java/org/apache/ratis/client/api/DataStreamOutput.java:
##########
@@ -37,7 +39,20 @@ public interface DataStreamOutput extends CloseAsync<DataStreamReply> {
    * @param options - options specifying how the data was written
    * @return a future of the reply.
    */
-  CompletableFuture<DataStreamReply> writeAsync(ByteBuffer src, WriteOption... options);
+  default CompletableFuture<DataStreamReply> writeAsync(ByteBuffer src,
+                                                 WriteOption... options) {

Review Comment:
   Please use a single line.  This line width in Ratis is 120.



##########
ratis-client/src/main/java/org/apache/ratis/client/impl/DataStreamClientImpl.java:
##########
@@ -153,12 +160,17 @@ private CompletableFuture<DataStreamReply> writeAsyncImpl(Object data, long leng
 
     @Override
     public CompletableFuture<DataStreamReply> writeAsync(ByteBuffer src, WriteOption... options) {
+      return writeAsync(src, Arrays.stream(options).collect(Collectors.toList()));

Review Comment:
   Use `Arrays.asList(options)`.



##########
ratis-common/src/main/java/org/apache/ratis/datastream/impl/DataStreamRequestByteBuffer.java:
##########
@@ -41,6 +41,7 @@ public DataStreamRequestByteBuffer(DataStreamRequestHeader header, ByteBuffer bu
 
   @Override
   @SuppressFBWarnings("EI_EXPOSE_REP")
+  @Deprecated

Review Comment:
   Change options to `List<WriteOption>`  and remove this method.



##########
ratis-client/src/main/java/org/apache/ratis/client/impl/OrderedStreamAsync.java:
##########
@@ -149,7 +149,8 @@ private void sendRequestToNetwork(DataStreamWindowRequest request,
         request.getDataStreamRequest());
     long seqNum = request.getSeqNum();
 
-    final boolean isClose = StandardWriteOption.CLOSE.isOneOf(request.getDataStreamRequest().getWriteOptions());
+    final boolean isClose =
+            StandardWriteOption.CLOSE.isOneOf(request.getDataStreamRequest().getWriteOptionsList());

Review Comment:
   Please use a single line. This line width in Ratis is 120.



##########
ratis-client/src/main/java/org/apache/ratis/client/impl/DataStreamClientImpl.java:
##########
@@ -123,10 +126,12 @@ private DataStreamOutputImpl(RaftClientRequest request) {
       this.header = request;
       this.slidingWindow = new SlidingWindow.Client<>(ClientInvocationId.valueOf(clientId, header.getCallId()));
       final ByteBuffer buffer = ClientProtoUtils.toRaftClientRequestProtoByteBuffer(header);
-      this.headerFuture = send(Type.STREAM_HEADER, buffer, buffer.remaining());
+      this.headerFuture = send(Type.STREAM_HEADER, buffer, buffer.remaining(),
+              Collections.EMPTY_LIST);

Review Comment:
   Use `Collections.emptyList()`; otherwise, there is a unchecked assignment warning.
   ```java
         this.headerFuture = send(Type.STREAM_HEADER, buffer, buffer.remaining(), Collections.emptyList());
   ```



##########
ratis-client/src/main/java/org/apache/ratis/client/impl/DataStreamClientImpl.java:
##########
@@ -123,10 +126,12 @@ private DataStreamOutputImpl(RaftClientRequest request) {
       this.header = request;
       this.slidingWindow = new SlidingWindow.Client<>(ClientInvocationId.valueOf(clientId, header.getCallId()));
       final ByteBuffer buffer = ClientProtoUtils.toRaftClientRequestProtoByteBuffer(header);
-      this.headerFuture = send(Type.STREAM_HEADER, buffer, buffer.remaining());
+      this.headerFuture = send(Type.STREAM_HEADER, buffer, buffer.remaining(),
+              Collections.EMPTY_LIST);
     }
-
-    private CompletableFuture<DataStreamReply> send(Type type, Object data, long length, WriteOption... options) {
+    private CompletableFuture<DataStreamReply> send(Type type, Object data,
+                                                    long length,
+                                                    Iterable<WriteOption> options) {

Review Comment:
   Please use a single line. This line width in Ratis is 120.



##########
ratis-client/src/main/java/org/apache/ratis/client/impl/DataStreamClientImpl.java:
##########
@@ -136,7 +141,9 @@ private CompletableFuture<DataStreamReply> combineHeader(CompletableFuture<DataS
       return future.thenCombine(headerFuture, (reply, headerReply) -> headerReply.isSuccess()? reply : headerReply);
     }
 
-    private CompletableFuture<DataStreamReply> writeAsyncImpl(Object data, long length, WriteOption... options) {
+    private CompletableFuture<DataStreamReply> writeAsyncImpl(Object data,
+                                               long length,
+                                               Iterable<WriteOption> options) {

Review Comment:
   Please use a single line. This line width in Ratis is 120.



##########
ratis-client/src/main/java/org/apache/ratis/client/api/DataStreamOutput.java:
##########
@@ -37,7 +39,20 @@ public interface DataStreamOutput extends CloseAsync<DataStreamReply> {
    * @param options - options specifying how the data was written
    * @return a future of the reply.
    */
-  CompletableFuture<DataStreamReply> writeAsync(ByteBuffer src, WriteOption... options);
+  default CompletableFuture<DataStreamReply> writeAsync(ByteBuffer src,
+                                                 WriteOption... options) {
+    return writeAsync(src, Arrays.asList(options));
+  }
+
+  /**
+   * Send out the data in the source buffer asynchronously.
+   *
+   * @param src the source buffer to be sent.
+   * @param options - options specifying how the data was written
+   * @return a future of the reply.
+   */
+  CompletableFuture<DataStreamReply> writeAsync(ByteBuffer src,
+                                                Iterable<WriteOption> options);

Review Comment:
   Please use a single line.  This line width in Ratis is 120.



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@ratis.apache.org

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


[GitHub] [ratis] szetszwo commented on pull request #716: RATIS-1647. Fix EI_EXPOSE_REP in DataStreamRequest implementations

Posted by GitBox <gi...@apache.org>.
szetszwo commented on PR #716:
URL: https://github.com/apache/ratis/pull/716#issuecomment-1250831344

   @swamirishi , please see https://issues.apache.org/jira/secure/attachment/13049441/716_review.patch for the suggested change.


-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@ratis.apache.org

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


[GitHub] [ratis] szetszwo commented on a diff in pull request #716: RATIS-1647. Fix EI_EXPOSE_REP in DataStreamRequest implementations

Posted by GitBox <gi...@apache.org>.
szetszwo commented on code in PR #716:
URL: https://github.com/apache/ratis/pull/716#discussion_r983215680


##########
ratis-common/src/main/java/org/apache/ratis/datastream/impl/DataStreamRequestByteBuffer.java:
##########
@@ -17,31 +17,31 @@
  */
 package org.apache.ratis.datastream.impl;
 
-import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
 import org.apache.ratis.io.WriteOption;
 import org.apache.ratis.protocol.DataStreamRequest;
 import org.apache.ratis.protocol.DataStreamRequestHeader;
+import org.apache.ratis.thirdparty.com.google.common.collect.Lists;
 import org.apache.ratis.util.Preconditions;
 
 import java.nio.ByteBuffer;
+import java.util.List;
 
 /**
  * Implements {@link DataStreamRequest} with {@link ByteBuffer}.
  *
  * This class is immutable.
  */
 public class DataStreamRequestByteBuffer extends DataStreamPacketByteBuffer implements DataStreamRequest {
-  private WriteOption[] options;
+  private List<WriteOption> options;
 
   public DataStreamRequestByteBuffer(DataStreamRequestHeader header, ByteBuffer buffer) {
     super(header.getClientId(), header.getType(), header.getStreamId(), header.getStreamOffset(), buffer);
-    this.options = header.getWriteOptions();
+    this.options = Lists.newArrayList(header.getWriteOptionsList());
     Preconditions.assertTrue(header.getDataLength() == buffer.remaining());
   }
 
   @Override
-  @SuppressFBWarnings("EI_EXPOSE_REP")
-  public WriteOption[] getWriteOptions() {
-    return options;
+  public List<WriteOption> getWriteOptionsList() {
+    return Lists.newArrayList(options);

Review Comment:
   Please make the list immutable and don't copy the list for each get operation.



##########
ratis-common/src/main/java/org/apache/ratis/protocol/DataStreamRequest.java:
##########
@@ -19,7 +19,9 @@
 package org.apache.ratis.protocol;
 
 import org.apache.ratis.io.WriteOption;
+import java.util.List;
 
 public interface DataStreamRequest extends DataStreamPacket {
-  WriteOption[] getWriteOptions();

Review Comment:
   This is a public interface.  We must first deprecate it before removing it later.



##########
ratis-common/src/main/java/org/apache/ratis/protocol/DataStreamRequest.java:
##########
@@ -19,7 +19,9 @@
 package org.apache.ratis.protocol;
 
 import org.apache.ratis.io.WriteOption;
+import java.util.List;
 
 public interface DataStreamRequest extends DataStreamPacket {
-  WriteOption[] getWriteOptions();
+
+  List<WriteOption> getWriteOptionsList();

Review Comment:
   Rename "getWriteOptionsList" to "getWriteOptionList".



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@ratis.apache.org

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


[GitHub] [ratis] szetszwo commented on a diff in pull request #716: RATIS-1647. Fix EI_EXPOSE_REP in DataStreamRequest implementations

Posted by GitBox <gi...@apache.org>.
szetszwo commented on code in PR #716:
URL: https://github.com/apache/ratis/pull/716#discussion_r947433212


##########
ratis-common/src/main/java/org/apache/ratis/protocol/DataStreamRequestHeader.java:
##########
@@ -37,8 +39,17 @@ public DataStreamRequestHeader(ClientId clientId, Type type, long streamId, long
     this.options = options;
   }
 
+  @SuppressFBWarnings("EI_EXPOSE_REP2")

Review Comment:
   Please update the constructor but not adding SuppressFBWarnings.



##########
ratis-client/src/main/java/org/apache/ratis/client/api/DataStreamOutput.java:
##########
@@ -39,6 +39,16 @@ public interface DataStreamOutput extends CloseAsync<DataStreamReply> {
    */
   CompletableFuture<DataStreamReply> writeAsync(ByteBuffer src, WriteOption... options);

Review Comment:
   Add the default implementation here.
   ```java
     /**
      * The same as writeAsync(src, Arrays.asList(options)).
      */
     default CompletableFuture<DataStreamReply> writeAsync(ByteBuffer src, WriteOption... options) {
       return writeAsync(src, Arrays.asList(options));
     }
   ```



##########
ratis-client/src/main/java/org/apache/ratis/client/impl/DataStreamClientImpl.java:
##########
@@ -126,7 +128,15 @@ private DataStreamOutputImpl(RaftClientRequest request) {
       this.headerFuture = send(Type.STREAM_HEADER, buffer, buffer.remaining());
     }
 
-    private CompletableFuture<DataStreamReply> send(Type type, Object data, long length, WriteOption... options) {
+    private CompletableFuture<DataStreamReply> send(Type type, Object data,
+                                                    long length,
+                                                    WriteOption... options) {
+      return this.send(type, data, length, Arrays.asList(options));
+    }

Review Comment:
   Remove this 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.

To unsubscribe, e-mail: issues-unsubscribe@ratis.apache.org

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