Skip to content

Commit 7b36305

Browse files
SteNicholasturboFei
authored andcommitted
[CELEBORN-2077] Improve toString by JEP-280 instead of ToStringBuilder
Improve `toString` by JEP-280 instead of `ToStringBuilder`. Since Java 9, String Concatenation has been handled better by default. ID | DESCRIPTION -- | -- JEP-280 | [Indify String Concatenation](https://openjdk.org/jeps/280) Backport apache/spark#51572. No. CI. Closes #3380 from SteNicholas/CELEBORN-2077. Authored-by: SteNicholas <programgeek@163.com> Signed-off-by: Wang, Fei <fwang12@ebay.com> (cherry picked from commit 66856f2) Signed-off-by: Wang, Fei <fwang12@ebay.com>
1 parent d42debd commit 7b36305

20 files changed

+112
-140
lines changed

common/src/main/java/org/apache/celeborn/common/network/buffer/FileSegmentManagedBuffer.java

Lines changed: 7 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -25,8 +25,6 @@
2525
import com.google.common.io.ByteStreams;
2626
import io.netty.channel.DefaultFileRegion;
2727
import io.netty.handler.stream.ChunkedStream;
28-
import org.apache.commons.lang3.builder.ToStringBuilder;
29-
import org.apache.commons.lang3.builder.ToStringStyle;
3028

3129
import org.apache.celeborn.common.network.util.LimitedInputStream;
3230
import org.apache.celeborn.common.network.util.TransportConf;
@@ -153,10 +151,12 @@ public long getLength() {
153151

154152
@Override
155153
public String toString() {
156-
return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE)
157-
.append("file", file)
158-
.append("offset", offset)
159-
.append("length", length)
160-
.toString();
154+
return "FileSegmentManagedBuffer[file="
155+
+ file
156+
+ ",offset="
157+
+ offset
158+
+ ",length="
159+
+ length
160+
+ "]";
161161
}
162162
}

common/src/main/java/org/apache/celeborn/common/network/buffer/NettyManagedBuffer.java

Lines changed: 1 addition & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -24,8 +24,6 @@
2424
import io.netty.buffer.ByteBuf;
2525
import io.netty.buffer.ByteBufInputStream;
2626
import io.netty.buffer.Unpooled;
27-
import org.apache.commons.lang3.builder.ToStringBuilder;
28-
import org.apache.commons.lang3.builder.ToStringStyle;
2927

3028
/** A {@link ManagedBuffer} backed by a Netty {@link ByteBuf}. */
3129
public class NettyManagedBuffer extends ManagedBuffer {
@@ -83,8 +81,6 @@ public Object convertToNettyForSsl() throws IOException {
8381

8482
@Override
8583
public String toString() {
86-
return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE)
87-
.append("buf", buf)
88-
.toString();
84+
return "NettyManagedBuffer[buf=" + buf + "]";
8985
}
9086
}

common/src/main/java/org/apache/celeborn/common/network/buffer/NioManagedBuffer.java

Lines changed: 1 addition & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -23,8 +23,6 @@
2323

2424
import io.netty.buffer.ByteBufInputStream;
2525
import io.netty.buffer.Unpooled;
26-
import org.apache.commons.lang3.builder.ToStringBuilder;
27-
import org.apache.commons.lang3.builder.ToStringStyle;
2826

2927
/** A {@link ManagedBuffer} backed by {@link ByteBuffer}. */
3028
public class NioManagedBuffer extends ManagedBuffer {
@@ -71,8 +69,6 @@ public Object convertToNettyForSsl() throws IOException {
7169

7270
@Override
7371
public String toString() {
74-
return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE)
75-
.append("buf", buf)
76-
.toString();
72+
return "NioManagedBuffer[buf=" + buf + "]";
7773
}
7874
}

common/src/main/java/org/apache/celeborn/common/network/client/TransportClient.java

Lines changed: 7 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -33,8 +33,6 @@
3333
import io.netty.channel.ChannelFuture;
3434
import io.netty.util.concurrent.Future;
3535
import io.netty.util.concurrent.GenericFutureListener;
36-
import org.apache.commons.lang3.builder.ToStringBuilder;
37-
import org.apache.commons.lang3.builder.ToStringStyle;
3836
import org.slf4j.Logger;
3937
import org.slf4j.LoggerFactory;
4038

@@ -368,11 +366,13 @@ public void close() {
368366

369367
@Override
370368
public String toString() {
371-
return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE)
372-
.append("remoteAddress", channel.remoteAddress())
373-
.append("clientId", clientId)
374-
.append("isActive", isActive())
375-
.toString();
369+
return "TransportClient[remoteAddress="
370+
+ channel.remoteAddress()
371+
+ "clientId="
372+
+ clientId
373+
+ ",isActive="
374+
+ isActive()
375+
+ "]";
376376
}
377377

378378
private static final AtomicLong counter = new AtomicLong();

common/src/main/java/org/apache/celeborn/common/network/protocol/ChunkFetchFailure.java

Lines changed: 5 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -20,8 +20,6 @@
2020
import java.util.Objects;
2121

2222
import io.netty.buffer.ByteBuf;
23-
import org.apache.commons.lang3.builder.ToStringBuilder;
24-
import org.apache.commons.lang3.builder.ToStringStyle;
2523

2624
import org.apache.celeborn.common.protocol.PbChunkFetchRequest;
2725

@@ -73,9 +71,10 @@ public boolean equals(Object other) {
7371

7472
@Override
7573
public String toString() {
76-
return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE)
77-
.append("streamChunkId", streamChunkSlice)
78-
.append("errorString", errorString)
79-
.toString();
74+
return "ChunkFetchFailure[streamChunkId="
75+
+ streamChunkSlice
76+
+ ",errorString="
77+
+ errorString
78+
+ "]";
8079
}
8180
}

common/src/main/java/org/apache/celeborn/common/network/protocol/ChunkFetchRequest.java

Lines changed: 1 addition & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -18,8 +18,6 @@
1818
package org.apache.celeborn.common.network.protocol;
1919

2020
import io.netty.buffer.ByteBuf;
21-
import org.apache.commons.lang3.builder.ToStringBuilder;
22-
import org.apache.commons.lang3.builder.ToStringStyle;
2321

2422
/**
2523
* Request to fetch a sequence of a single chunk of a stream. This will correspond to a single
@@ -68,8 +66,6 @@ public boolean equals(Object other) {
6866

6967
@Override
7068
public String toString() {
71-
return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE)
72-
.append("streamChunkId", streamChunkSlice)
73-
.toString();
69+
return "ChunkFetchRequest[streamChunkId=" + streamChunkSlice + "]";
7470
}
7571
}

common/src/main/java/org/apache/celeborn/common/network/protocol/ChunkFetchSuccess.java

Lines changed: 1 addition & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -20,8 +20,6 @@
2020
import java.util.Objects;
2121

2222
import io.netty.buffer.ByteBuf;
23-
import org.apache.commons.lang3.builder.ToStringBuilder;
24-
import org.apache.commons.lang3.builder.ToStringStyle;
2523

2624
import org.apache.celeborn.common.network.buffer.ManagedBuffer;
2725
import org.apache.celeborn.common.network.buffer.NettyManagedBuffer;
@@ -93,9 +91,6 @@ public boolean equals(Object other) {
9391

9492
@Override
9593
public String toString() {
96-
return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE)
97-
.append("streamChunkId", streamChunkSlice)
98-
.append("buffer", body())
99-
.toString();
94+
return "ChunkFetchSuccess[streamChunkId=" + streamChunkSlice + ",body=" + body() + "]";
10095
}
10196
}

common/src/main/java/org/apache/celeborn/common/network/protocol/OneWayMessage.java

Lines changed: 1 addition & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -20,8 +20,6 @@
2020
import java.util.Objects;
2121

2222
import io.netty.buffer.ByteBuf;
23-
import org.apache.commons.lang3.builder.ToStringBuilder;
24-
import org.apache.commons.lang3.builder.ToStringStyle;
2523

2624
import org.apache.celeborn.common.network.buffer.ManagedBuffer;
2725
import org.apache.celeborn.common.network.buffer.NettyManagedBuffer;
@@ -83,8 +81,6 @@ public boolean equals(Object other) {
8381

8482
@Override
8583
public String toString() {
86-
return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE)
87-
.append("body", body())
88-
.toString();
84+
return "OneWayMessage[body=" + body() + "]";
8985
}
9086
}

common/src/main/java/org/apache/celeborn/common/network/protocol/OpenStream.java

Lines changed: 9 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -22,8 +22,6 @@
2222
import java.util.Objects;
2323

2424
import io.netty.buffer.ByteBuf;
25-
import org.apache.commons.lang3.builder.ToStringBuilder;
26-
import org.apache.commons.lang3.builder.ToStringStyle;
2725

2826
/**
2927
* Request to read a set of blocks. Returns {@link StreamHandle}. Use PbOpenStream instead of this
@@ -101,11 +99,14 @@ public boolean equals(Object other) {
10199

102100
@Override
103101
public String toString() {
104-
return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE)
105-
.append("shuffleKey", new String(shuffleKey, StandardCharsets.UTF_8))
106-
.append("fileName", new String(fileName, StandardCharsets.UTF_8))
107-
.append("startMapIndex", startMapIndex)
108-
.append("endMapIndex", endMapIndex)
109-
.toString();
102+
return "OpenStream[shuffleKey="
103+
+ new String(shuffleKey, StandardCharsets.UTF_8)
104+
+ ",fileName="
105+
+ new String(fileName, StandardCharsets.UTF_8)
106+
+ ",startMapIndex="
107+
+ startMapIndex
108+
+ ",endMapIndex="
109+
+ endMapIndex
110+
+ "]";
110111
}
111112
}

common/src/main/java/org/apache/celeborn/common/network/protocol/PushData.java

Lines changed: 11 additions & 9 deletions
Original file line numberDiff line numberDiff line change
@@ -20,8 +20,6 @@
2020
import java.util.Objects;
2121

2222
import io.netty.buffer.ByteBuf;
23-
import org.apache.commons.lang3.builder.ToStringBuilder;
24-
import org.apache.commons.lang3.builder.ToStringStyle;
2523

2624
import org.apache.celeborn.common.network.buffer.ManagedBuffer;
2725
import org.apache.celeborn.common.network.buffer.NettyManagedBuffer;
@@ -107,12 +105,16 @@ public boolean equals(Object other) {
107105

108106
@Override
109107
public String toString() {
110-
return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE)
111-
.append("requestId", requestId)
112-
.append("mode", mode)
113-
.append("shuffleKey", shuffleKey)
114-
.append("partitionUniqueId", partitionUniqueId)
115-
.append("body size", body().size())
116-
.toString();
108+
return "PushData[requestId="
109+
+ requestId
110+
+ ",mode="
111+
+ mode
112+
+ ",shuffleKey="
113+
+ shuffleKey
114+
+ ",partitionUniqueId="
115+
+ partitionUniqueId
116+
+ ",body size="
117+
+ body().size()
118+
+ "]";
117119
}
118120
}

0 commit comments

Comments
 (0)