Skip to content

Commit 2298963

Browse files
authored
HDDS-11665. Minor optimizations on the write path (apache#7407)
(cherry picked from commit 5663971)
1 parent 30798af commit 2298963

4 files changed

Lines changed: 5 additions & 11 deletions

File tree

hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientRatis.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -282,7 +282,7 @@ private CompletableFuture<RaftClientReply> sendRequestAsync(
282282
// gets the minimum log index replicated to all servers
283283
@Override
284284
public long getReplicatedMinCommitIndex() {
285-
return commitInfoMap.values().parallelStream()
285+
return commitInfoMap.values().stream()
286286
.mapToLong(Long::longValue).min().orElse(0);
287287
}
288288

hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/ChunkBufferImplWithByteBuffer.java

Lines changed: 1 addition & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -25,7 +25,6 @@
2525
import java.util.List;
2626
import java.util.NoSuchElementException;
2727
import java.util.Objects;
28-
import java.util.UUID;
2928
import java.util.function.Function;
3029

3130
import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
@@ -35,7 +34,6 @@
3534
final class ChunkBufferImplWithByteBuffer implements ChunkBuffer {
3635
private final ByteBuffer buffer;
3736
private final UncheckedAutoCloseable underlying;
38-
private final UUID identity = UUID.randomUUID();
3937

4038
ChunkBufferImplWithByteBuffer(ByteBuffer buffer) {
4139
this(buffer, null);
@@ -163,6 +161,6 @@ public int hashCode() {
163161
@Override
164162
public String toString() {
165163
return getClass().getSimpleName() + ":limit=" + buffer.limit()
166-
+ "@" + identity;
164+
+ "@" + Integer.toHexString(super.hashCode());
167165
}
168166
}

hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/container/common/helpers/BlockData.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -254,7 +254,7 @@ public void setChunks(List<ContainerProtos.ChunkInfo> chunks) {
254254
size = singleChunk.getLen();
255255
} else {
256256
chunkList = chunks;
257-
size = chunks.parallelStream()
257+
size = chunks.stream()
258258
.mapToLong(ContainerProtos.ChunkInfo::getLen)
259259
.sum();
260260
}

hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/KeyOutputStream.java

Lines changed: 2 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -303,7 +303,7 @@ private int writeToOutputStream(BlockOutputStreamEntry current,
303303
if (retry) {
304304
current.writeOnRetry(len);
305305
} else {
306-
waitForRetryHandling(current);
306+
current.waitForRetryHandling(retryHandlingCondition);
307307
current.write(b, off, writeLen);
308308
offset += writeLen;
309309
}
@@ -584,7 +584,7 @@ private void handleFlushOrClose(StreamAction op) throws IOException {
584584
blockOutputStreamEntryPool.getCurrentStreamEntry();
585585
if (entry != null) {
586586
// If the current block is to handle retries, wait until all the retries are done.
587-
waitForRetryHandling(entry);
587+
doInWriteLock(() -> entry.waitForRetryHandling(retryHandlingCondition));
588588
entry.registerCallReceived();
589589
try {
590590
handleStreamAction(entry, op);
@@ -608,10 +608,6 @@ private void handleFlushOrClose(StreamAction op) throws IOException {
608608
}
609609
}
610610

611-
private void waitForRetryHandling(BlockOutputStreamEntry currentEntry) throws InterruptedException {
612-
doInWriteLock(() -> currentEntry.waitForRetryHandling(retryHandlingCondition));
613-
}
614-
615611
private void handleStreamAction(BlockOutputStreamEntry entry,
616612
StreamAction op) throws IOException {
617613
Collection<DatanodeDetails> failedServers = entry.getFailedServers();

0 commit comments

Comments
 (0)