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 io.netty.buffer.ByteBuf;
0021 import org.apache.commons.lang3.builder.ToStringBuilder;
0022 import org.apache.commons.lang3.builder.ToStringStyle;
0023 
0024 /**
0025  * Request to fetch a sequence of a single chunk of a stream. This will correspond to a single
0026  * {@link org.apache.spark.network.protocol.ResponseMessage} (either success or failure).
0027  */
0028 public final class ChunkFetchRequest extends AbstractMessage implements RequestMessage {
0029   public final StreamChunkId streamChunkId;
0030 
0031   public ChunkFetchRequest(StreamChunkId streamChunkId) {
0032     this.streamChunkId = streamChunkId;
0033   }
0034 
0035   @Override
0036   public Message.Type type() { return Type.ChunkFetchRequest; }
0037 
0038   @Override
0039   public int encodedLength() {
0040     return streamChunkId.encodedLength();
0041   }
0042 
0043   @Override
0044   public void encode(ByteBuf buf) {
0045     streamChunkId.encode(buf);
0046   }
0047 
0048   public static ChunkFetchRequest decode(ByteBuf buf) {
0049     return new ChunkFetchRequest(StreamChunkId.decode(buf));
0050   }
0051 
0052   @Override
0053   public int hashCode() {
0054     return streamChunkId.hashCode();
0055   }
0056 
0057   @Override
0058   public boolean equals(Object other) {
0059     if (other instanceof ChunkFetchRequest) {
0060       ChunkFetchRequest o = (ChunkFetchRequest) other;
0061       return streamChunkId.equals(o.streamChunkId);
0062     }
0063     return false;
0064   }
0065 
0066   @Override
0067   public String toString() {
0068     return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE)
0069       .append("streamChunkId", streamChunkId)
0070       .toString();
0071   }
0072 }