blob: 26747ee55b4dea93c1f8eb5cbb23c239c80506de (
plain) (
blame)
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
|
/*
* 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 com.google.common.base.Objects;
import io.netty.buffer.ByteBuf;
import org.apache.spark.network.buffer.ManagedBuffer;
import org.apache.spark.network.buffer.NettyManagedBuffer;
/**
* Message indicating an error when transferring a stream.
*/
public final class StreamFailure extends AbstractMessage implements ResponseMessage {
public final String streamId;
public final String error;
public StreamFailure(String streamId, String error) {
this.streamId = streamId;
this.error = error;
}
@Override
public Type type() { return Type.StreamFailure; }
@Override
public int encodedLength() {
return Encoders.Strings.encodedLength(streamId) + Encoders.Strings.encodedLength(error);
}
@Override
public void encode(ByteBuf buf) {
Encoders.Strings.encode(buf, streamId);
Encoders.Strings.encode(buf, error);
}
public static StreamFailure decode(ByteBuf buf) {
String streamId = Encoders.Strings.decode(buf);
String error = Encoders.Strings.decode(buf);
return new StreamFailure(streamId, error);
}
@Override
public int hashCode() {
return Objects.hashCode(streamId, error);
}
@Override
public boolean equals(Object other) {
if (other instanceof StreamFailure) {
StreamFailure o = (StreamFailure) other;
return streamId.equals(o.streamId) && error.equals(o.error);
}
return false;
}
@Override
public String toString() {
return Objects.toStringHelper(this)
.add("streamId", streamId)
.add("error", error)
.toString();
}
}
|