Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -29,8 +29,6 @@
import com.google.common.io.ByteStreams;
import io.netty.channel.DefaultFileRegion;
import io.netty.handler.stream.ChunkedStream;
import org.apache.commons.lang3.builder.ToStringBuilder;
import org.apache.commons.lang3.builder.ToStringStyle;

import org.apache.spark.network.util.JavaUtils;
import org.apache.spark.network.util.LimitedInputStream;
Expand Down Expand Up @@ -152,10 +150,7 @@ public Object convertToNettyForSsl() throws IOException {

@Override
public String toString() {
return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE)
.append("file", file)
.append("offset", offset)
.append("length", length)
.toString();
return "FileSegmentManagedBuffer[file=" + file + ",offset=" + offset +
",length=" + length + "]";
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -23,8 +23,6 @@

import io.netty.buffer.ByteBuf;
import io.netty.buffer.ByteBufInputStream;
import org.apache.commons.lang3.builder.ToStringBuilder;
import org.apache.commons.lang3.builder.ToStringStyle;

/**
* A {@link ManagedBuffer} backed by a Netty {@link ByteBuf}.
Expand Down Expand Up @@ -75,8 +73,6 @@ public Object convertToNettyForSsl() throws IOException {

@Override
public String toString() {
return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE)
.append("buf", buf)
.toString();
return "NettyManagedBuffer[buf=" + buf + "]";
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -23,8 +23,6 @@

import io.netty.buffer.ByteBufInputStream;
import io.netty.buffer.Unpooled;
import org.apache.commons.lang3.builder.ToStringBuilder;
import org.apache.commons.lang3.builder.ToStringStyle;

/**
* A {@link ManagedBuffer} backed by {@link ByteBuffer}.
Expand Down Expand Up @@ -73,9 +71,7 @@ public Object convertToNettyForSsl() throws IOException {

@Override
public String toString() {
return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE)
.append("buf", buf)
.toString();
return "NioManagedBuffer[buf=" + buf + "]";
}
}

Original file line number Diff line number Diff line change
Expand Up @@ -33,8 +33,6 @@
import io.netty.channel.Channel;
import io.netty.util.concurrent.Future;
import io.netty.util.concurrent.GenericFutureListener;
import org.apache.commons.lang3.builder.ToStringBuilder;
import org.apache.commons.lang3.builder.ToStringStyle;

import org.apache.spark.internal.SparkLogger;
import org.apache.spark.internal.SparkLoggerFactory;
Expand Down Expand Up @@ -338,11 +336,8 @@ public void close() {

@Override
public String toString() {
return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE)
.append("remoteAddress", channel.remoteAddress())
.append("clientId", clientId)
.append("isActive", isActive())
.toString();
return "TransportClient[remoteAddress=" + channel.remoteAddress() + "clientId=" + clientId +
",isActive=" + isActive() + "]";
}

private static long requestId() {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,8 +20,6 @@
import java.util.Objects;

import io.netty.buffer.ByteBuf;
import org.apache.commons.lang3.builder.ToStringBuilder;
import org.apache.commons.lang3.builder.ToStringStyle;

/**
* Response to {@link ChunkFetchRequest} when there is an error fetching the chunk.
Expand Down Expand Up @@ -70,9 +68,6 @@ public boolean equals(Object other) {

@Override
public String toString() {
return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE)
.append("streamChunkId", streamChunkId)
.append("errorString", errorString)
.toString();
return "ChunkFetchFailure[streamChunkId=" + streamChunkId + ",errorString=" + errorString + "]";
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -18,8 +18,6 @@
package org.apache.spark.network.protocol;

import io.netty.buffer.ByteBuf;
import org.apache.commons.lang3.builder.ToStringBuilder;
import org.apache.commons.lang3.builder.ToStringStyle;

/**
* Request to fetch a sequence of a single chunk of a stream. This will correspond to a single
Expand Down Expand Up @@ -64,8 +62,6 @@ public boolean equals(Object other) {

@Override
public String toString() {
return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE)
.append("streamChunkId", streamChunkId)
.toString();
return "ChunkFetchRequest[streamChunkId=" + streamChunkId + "]";
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -20,8 +20,6 @@
import java.util.Objects;

import io.netty.buffer.ByteBuf;
import org.apache.commons.lang3.builder.ToStringBuilder;
import org.apache.commons.lang3.builder.ToStringStyle;

import org.apache.spark.network.buffer.ManagedBuffer;
import org.apache.spark.network.buffer.NettyManagedBuffer;
Expand Down Expand Up @@ -83,9 +81,6 @@ public boolean equals(Object other) {

@Override
public String toString() {
return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE)
.append("streamChunkId", streamChunkId)
.append("body", body())
.toString();
return "ChunkFetchSuccess[streamChunkId=" + streamChunkId + ",body=" + body() + "]";
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -19,8 +19,6 @@

import com.google.common.base.Objects;
import io.netty.buffer.ByteBuf;
import org.apache.commons.lang3.builder.ToStringBuilder;
import org.apache.commons.lang3.builder.ToStringStyle;

/**
* Request to find the meta information for the specified merged block. The meta information
Expand Down Expand Up @@ -94,12 +92,7 @@ public boolean equals(Object other) {

@Override
public String toString() {
return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE)
.append("requestId", requestId)
.append("appId", appId)
.append("shuffleId", shuffleId)
.append("shuffleMergeId", shuffleMergeId)
.append("reduceId", reduceId)
.toString();
return "MergedBlockMetaRequest[requestId=" + requestId + ",appId=" + appId + ",shuffleId=" +
shuffleId + ",shuffleMergeId=" + shuffleMergeId + ",reduceId=" + reduceId + "]";
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -19,8 +19,6 @@

import com.google.common.base.Objects;
import io.netty.buffer.ByteBuf;
import org.apache.commons.lang3.builder.ToStringBuilder;
import org.apache.commons.lang3.builder.ToStringStyle;

import org.apache.spark.network.buffer.ManagedBuffer;
import org.apache.spark.network.buffer.NettyManagedBuffer;
Expand Down Expand Up @@ -56,8 +54,7 @@ public int hashCode() {

@Override
public String toString() {
return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE)
.append("requestId", requestId).append("numChunks", numChunks).toString();
return "MergedBlockMetaSuccess[requestId=" + requestId + ",numChunks=" + numChunks + "]";
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,8 +20,6 @@
import java.util.Objects;

import io.netty.buffer.ByteBuf;
import org.apache.commons.lang3.builder.ToStringBuilder;
import org.apache.commons.lang3.builder.ToStringStyle;

import org.apache.spark.network.buffer.ManagedBuffer;
import org.apache.spark.network.buffer.NettyManagedBuffer;
Expand Down Expand Up @@ -74,8 +72,6 @@ public boolean equals(Object other) {

@Override
public String toString() {
return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE)
.append("body", body())
.toString();
return "OneWayMessage[body=" + body() + "]";
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -20,8 +20,6 @@
import java.util.Objects;

import io.netty.buffer.ByteBuf;
import org.apache.commons.lang3.builder.ToStringBuilder;
import org.apache.commons.lang3.builder.ToStringStyle;

/** Response to {@link RpcRequest} for a failed RPC. */
public final class RpcFailure extends AbstractMessage implements ResponseMessage {
Expand Down Expand Up @@ -68,9 +66,6 @@ public boolean equals(Object other) {

@Override
public String toString() {
return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE)
.append("requestId", requestId)
.append("errorString", errorString)
.toString();
return "RpcFailure[requestId=" + requestId + ",errorString=" + errorString + "]";
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -20,8 +20,6 @@
import java.util.Objects;

import io.netty.buffer.ByteBuf;
import org.apache.commons.lang3.builder.ToStringBuilder;
import org.apache.commons.lang3.builder.ToStringStyle;

import org.apache.spark.network.buffer.ManagedBuffer;
import org.apache.spark.network.buffer.NettyManagedBuffer;
Expand Down Expand Up @@ -80,9 +78,6 @@ public boolean equals(Object other) {

@Override
public String toString() {
return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE)
.append("requestId", requestId)
.append("body", body())
.toString();
return "RpcRequest[requestId=" + requestId + ",body=" + body() + "]";
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -20,8 +20,6 @@
import java.util.Objects;

import io.netty.buffer.ByteBuf;
import org.apache.commons.lang3.builder.ToStringBuilder;
import org.apache.commons.lang3.builder.ToStringStyle;

import org.apache.spark.network.buffer.ManagedBuffer;
import org.apache.spark.network.buffer.NettyManagedBuffer;
Expand Down Expand Up @@ -80,9 +78,6 @@ public boolean equals(Object other) {

@Override
public String toString() {
return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE)
.append("requestId", requestId)
.append("body", body())
.toString();
return "RpcResponse[requestId=" + requestId + ",body=" + body() + "]";
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -20,8 +20,6 @@
import java.util.Objects;

import io.netty.buffer.ByteBuf;
import org.apache.commons.lang3.builder.ToStringBuilder;
import org.apache.commons.lang3.builder.ToStringStyle;

/**
* Encapsulates a request for a particular chunk of a stream.
Expand Down Expand Up @@ -61,9 +59,6 @@ public boolean equals(Object other) {

@Override
public String toString() {
return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE)
.append("streamId", streamId)
.append("chunkIndex", chunkIndex)
.toString();
return "StreamChunkId[streamId=" + streamId + ",chunkIndex=" + chunkIndex + "]";
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -20,8 +20,6 @@
import java.util.Objects;

import io.netty.buffer.ByteBuf;
import org.apache.commons.lang3.builder.ToStringBuilder;
import org.apache.commons.lang3.builder.ToStringStyle;

/**
* Message indicating an error when transferring a stream.
Expand Down Expand Up @@ -70,10 +68,7 @@ public boolean equals(Object other) {

@Override
public String toString() {
return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE)
.append("streamId", streamId)
.append("error", error)
.toString();
return "StreamFailure[streamId=" + streamId + ",error=" + error + "]";
}

}
Original file line number Diff line number Diff line change
Expand Up @@ -20,8 +20,6 @@
import java.util.Objects;

import io.netty.buffer.ByteBuf;
import org.apache.commons.lang3.builder.ToStringBuilder;
import org.apache.commons.lang3.builder.ToStringStyle;

/**
* Request to stream data from the remote end.
Expand Down Expand Up @@ -69,9 +67,7 @@ public boolean equals(Object other) {

@Override
public String toString() {
return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE)
.append("streamId", streamId)
.toString();
return "StreamRequest[streamId=" + streamId + "]";
}

}
Original file line number Diff line number Diff line change
Expand Up @@ -20,8 +20,6 @@
import java.util.Objects;

import io.netty.buffer.ByteBuf;
import org.apache.commons.lang3.builder.ToStringBuilder;
import org.apache.commons.lang3.builder.ToStringStyle;

import org.apache.spark.network.buffer.ManagedBuffer;

Expand Down Expand Up @@ -83,11 +81,8 @@ public boolean equals(Object other) {

@Override
public String toString() {
return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE)
.append("streamId", streamId)
.append("byteCount", byteCount)
.append("body", body())
.toString();
return "StreamResponse[streamId=" + streamId + ",byteCount=" + byteCount +
",body=" + body() + "]";
}

}
Original file line number Diff line number Diff line change
Expand Up @@ -21,8 +21,6 @@
import java.nio.ByteBuffer;

import io.netty.buffer.ByteBuf;
import org.apache.commons.lang3.builder.ToStringBuilder;
import org.apache.commons.lang3.builder.ToStringStyle;

import org.apache.spark.network.buffer.ManagedBuffer;
import org.apache.spark.network.buffer.NettyManagedBuffer;
Expand Down Expand Up @@ -99,9 +97,6 @@ public boolean equals(Object other) {

@Override
public String toString() {
return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE)
.append("requestId", requestId)
.append("body", body())
.toString();
return "UploadStream[requestId=" + requestId + ",body=" + body() + "]";
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -26,8 +26,6 @@
import java.util.concurrent.Executors;
import java.util.stream.Collectors;

import org.apache.commons.lang3.builder.ToStringBuilder;
import org.apache.commons.lang3.builder.ToStringStyle;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.databind.ObjectMapper;
Expand Down Expand Up @@ -451,10 +449,7 @@ public int hashCode() {

@Override
public String toString() {
return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE)
.append("appId", appId)
.append("execId", execId)
.toString();
return "ExternalShuffleBlockResolver[appId=" + appId + ",execId=" + execId + "]";
}
}

Expand Down
Loading