Back to home page

OSCL-LXR

 
 

    


0001 /*
0002  * Licensed to the Apache Software Foundation (ASF) under one or more
0003  * contributor license agreements.  See the NOTICE file distributed with
0004  * this work for additional information regarding copyright ownership.
0005  * The ASF licenses this file to You under the Apache License, Version 2.0
0006  * (the "License"); you may not use this file except in compliance with
0007  * the License.  You may obtain a copy of the License at
0008  *
0009  *    http://www.apache.org/licenses/LICENSE-2.0
0010  *
0011  * Unless required by applicable law or agreed to in writing, software
0012  * distributed under the License is distributed on an "AS IS" BASIS,
0013  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
0014  * See the License for the specific language governing permissions and
0015  * limitations under the License.
0016  */
0017 
0018 package org.apache.spark.network.protocol;
0019 
0020 import java.util.Objects;
0021 
0022 import io.netty.buffer.ByteBuf;
0023 import org.apache.commons.lang3.builder.ToStringBuilder;
0024 import org.apache.commons.lang3.builder.ToStringStyle;
0025 
0026 import org.apache.spark.network.buffer.ManagedBuffer;
0027 import org.apache.spark.network.buffer.NettyManagedBuffer;
0028 
0029 /**
0030  * Response to {@link ChunkFetchRequest} when a chunk exists and has been successfully fetched.
0031  *
0032  * Note that the server-side encoding of this messages does NOT include the buffer itself, as this
0033  * may be written by Netty in a more efficient manner (i.e., zero-copy write).
0034  * Similarly, the client-side decoding will reuse the Netty ByteBuf as the buffer.
0035  */
0036 public final class ChunkFetchSuccess extends AbstractResponseMessage {
0037   public final StreamChunkId streamChunkId;
0038 
0039   public ChunkFetchSuccess(StreamChunkId streamChunkId, ManagedBuffer buffer) {
0040     super(buffer, true);
0041     this.streamChunkId = streamChunkId;
0042   }
0043 
0044   @Override
0045   public Message.Type type() { return Type.ChunkFetchSuccess; }
0046 
0047   @Override
0048   public int encodedLength() {
0049     return streamChunkId.encodedLength();
0050   }
0051 
0052   /** Encoding does NOT include 'buffer' itself. See {@link MessageEncoder}. */
0053   @Override
0054   public void encode(ByteBuf buf) {
0055     streamChunkId.encode(buf);
0056   }
0057 
0058   @Override
0059   public ResponseMessage createFailureResponse(String error) {
0060     return new ChunkFetchFailure(streamChunkId, error);
0061   }
0062 
0063   /** Decoding uses the given ByteBuf as our data, and will retain() it. */
0064   public static ChunkFetchSuccess decode(ByteBuf buf) {
0065     StreamChunkId streamChunkId = StreamChunkId.decode(buf);
0066     buf.retain();
0067     NettyManagedBuffer managedBuf = new NettyManagedBuffer(buf.duplicate());
0068     return new ChunkFetchSuccess(streamChunkId, managedBuf);
0069   }
0070 
0071   @Override
0072   public int hashCode() {
0073     return Objects.hash(streamChunkId, body());
0074   }
0075 
0076   @Override
0077   public boolean equals(Object other) {
0078     if (other instanceof ChunkFetchSuccess) {
0079       ChunkFetchSuccess o = (ChunkFetchSuccess) other;
0080       return streamChunkId.equals(o.streamChunkId) && super.equals(o);
0081     }
0082     return false;
0083   }
0084 
0085   @Override
0086   public String toString() {
0087     return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE)
0088       .append("streamChunkId", streamChunkId)
0089       .append("buffer", body())
0090       .toString();
0091   }
0092 }