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 /** Response to {@link RpcRequest} for a successful RPC. */
0030 public final class RpcResponse extends AbstractResponseMessage {
0031   public final long requestId;
0032 
0033   public RpcResponse(long requestId, ManagedBuffer message) {
0034     super(message, true);
0035     this.requestId = requestId;
0036   }
0037 
0038   @Override
0039   public Message.Type type() { return Type.RpcResponse; }
0040 
0041   @Override
0042   public int encodedLength() {
0043     // The integer (a.k.a. the body size) is not really used, since that information is already
0044     // encoded in the frame length. But this maintains backwards compatibility with versions of
0045     // RpcRequest that use Encoders.ByteArrays.
0046     return 8 + 4;
0047   }
0048 
0049   @Override
0050   public void encode(ByteBuf buf) {
0051     buf.writeLong(requestId);
0052     // See comment in encodedLength().
0053     buf.writeInt((int) body().size());
0054   }
0055 
0056   @Override
0057   public ResponseMessage createFailureResponse(String error) {
0058     return new RpcFailure(requestId, error);
0059   }
0060 
0061   public static RpcResponse decode(ByteBuf buf) {
0062     long requestId = buf.readLong();
0063     // See comment in encodedLength().
0064     buf.readInt();
0065     return new RpcResponse(requestId, new NettyManagedBuffer(buf.retain()));
0066   }
0067 
0068   @Override
0069   public int hashCode() {
0070     return Objects.hash(requestId, body());
0071   }
0072 
0073   @Override
0074   public boolean equals(Object other) {
0075     if (other instanceof RpcResponse) {
0076       RpcResponse o = (RpcResponse) other;
0077       return requestId == o.requestId && super.equals(o);
0078     }
0079     return false;
0080   }
0081 
0082   @Override
0083   public String toString() {
0084     return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE)
0085       .append("requestId", requestId)
0086       .append("body", body())
0087       .toString();
0088   }
0089 }