hasnain-db commented on code in PR #43244:
URL: https://github.com/apache/spark/pull/43244#discussion_r1353972708


##########
common/network-common/src/main/java/org/apache/spark/network/protocol/EncryptedMessageWithHeader.java:
##########
@@ -0,0 +1,132 @@
+
+/*
+ * 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.spark.network.protocol;
+
+import javax.annotation.Nullable;
+import java.io.InputStream;
+
+import com.google.common.base.Preconditions;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.ByteBufAllocator;
+import io.netty.channel.ChannelHandlerContext;
+import io.netty.handler.stream.ChunkedStream;
+import io.netty.handler.stream.ChunkedInput;
+
+import org.apache.spark.network.buffer.ManagedBuffer;
+
+/**
+ * A wrapper message that holds two separate pieces (a header and a body).
+ *
+ * The header must be a ByteBuf, while the body can be any InputStream or 
ChunkedStream
+ */
+public class EncryptedMessageWithHeader implements ChunkedInput<ByteBuf> {
+
+  @Nullable private final ManagedBuffer managedBuffer;
+  private final ByteBuf header;
+  private final int headerLength;
+  private final Object body;
+  private final long bodyLength;
+  private long totalBytesTransferred;
+
+  /**
+   * Construct a new EncryptedMessageWithHeader.
+   *
+   * @param managedBuffer the {@link ManagedBuffer} that the message body came 
from. This needs to
+   *                      be passed in so that the buffer can be freed when 
this message is
+   *                      deallocated. Ownership of the caller's reference to 
this buffer is
+   *                      transferred to this class, so if the caller wants to 
continue to use the
+   *                      ManagedBuffer in other messages then they will need 
to call retain() on
+   *                      it before passing it to this constructor.
+   * @param header the message header.
+   * @param body the message body.
+   * @param bodyLength the length of the message body, in bytes.
+   */
+
+  public EncryptedMessageWithHeader(
+      @Nullable ManagedBuffer managedBuffer, ByteBuf header, Object body, long 
bodyLength) {
+    Preconditions.checkArgument(body instanceof InputStream || body instanceof 
ChunkedStream,
+      "Body must be an InputStream or a ChunkedStream.");
+    this.managedBuffer = managedBuffer;
+    this.header = header;
+    this.headerLength = header.readableBytes();
+    this.body = body;
+    this.bodyLength = bodyLength;
+    this.totalBytesTransferred = 0;
+  }
+
+  @Override
+  public ByteBuf readChunk(ChannelHandlerContext ctx) throws Exception {
+    return readChunk(ctx.alloc());
+  }
+
+  @Override
+  public ByteBuf readChunk(ByteBufAllocator allocator) throws Exception {
+    if (isEndOfInput()) {
+      return null;
+    }
+
+    if (totalBytesTransferred < headerLength) {
+      totalBytesTransferred += headerLength;
+      return header.retain();
+    } else if (body instanceof InputStream) {
+      InputStream stream = (InputStream) body;
+      int available = stream.available();

Review Comment:
   @mridulm I think I'm confused a little (probably missing something).
   
   Inside our implementation of `readBuf`, we need to return a `ByteBuf` to the 
caller. When we're backed by an `InputStream`, we have to allocate a `ByteBuf` 
ourselves, sizing it appropriately, and asking the `InputStream` to write bytes 
into it. We thus have to properly account for how much is available, how much 
we ask for, and how much is read. I've made the changes as you suggested 
(correctly, I hope) - as this makes sense.
   
   However, when we're backed by a `ChunkStream`, the only API it gives us is a 
`readChunk` - that returns a `ByteBuf` directly and so we just need to count 
how many new bytes were transferred and return the buffer directly. Am I 
missing something?



-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to