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 
0028 /**
0029  * Response to {@link StreamRequest} when the stream has been successfully opened.
0030  * <p>
0031  * Note the message itself does not contain the stream data. That is written separately by the
0032  * sender. The receiver is expected to set a temporary channel handler that will consume the
0033  * number of bytes this message says the stream has.
0034  */
0035 public final class StreamResponse extends AbstractResponseMessage {
0036   public final String streamId;
0037   public final long byteCount;
0038 
0039   public StreamResponse(String streamId, long byteCount, ManagedBuffer buffer) {
0040     super(buffer, false);
0041     this.streamId = streamId;
0042     this.byteCount = byteCount;
0043   }
0044 
0045   @Override
0046   public Message.Type type() { return Type.StreamResponse; }
0047 
0048   @Override
0049   public int encodedLength() {
0050     return 8 + Encoders.Strings.encodedLength(streamId);
0051   }
0052 
0053   /** Encoding does NOT include 'buffer' itself. See {@link MessageEncoder}. */
0054   @Override
0055   public void encode(ByteBuf buf) {
0056     Encoders.Strings.encode(buf, streamId);
0057     buf.writeLong(byteCount);
0058   }
0059 
0060   @Override
0061   public ResponseMessage createFailureResponse(String error) {
0062     return new StreamFailure(streamId, error);
0063   }
0064 
0065   public static StreamResponse decode(ByteBuf buf) {
0066     String streamId = Encoders.Strings.decode(buf);
0067     long byteCount = buf.readLong();
0068     return new StreamResponse(streamId, byteCount, null);
0069   }
0070 
0071   @Override
0072   public int hashCode() {
0073     return Objects.hash(byteCount, streamId);
0074   }
0075 
0076   @Override
0077   public boolean equals(Object other) {
0078     if (other instanceof StreamResponse) {
0079       StreamResponse o = (StreamResponse) other;
0080       return byteCount == o.byteCount && streamId.equals(o.streamId);
0081     }
0082     return false;
0083   }
0084 
0085   @Override
0086   public String toString() {
0087     return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE)
0088       .append("streamId", streamId)
0089       .append("byteCount", byteCount)
0090       .append("body", body())
0091       .toString();
0092   }
0093 
0094 }