Skip to content
Open
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 @@ -74,14 +74,6 @@ public final class ECKeyOutputStream extends KeyOutputStream
private final Future<Boolean> flushFuture;
private final AtomicLong flushCheckpoint;

/**
* Indicates if an atomic write is required. When set to true,
* the amount of data written must match the declared size during the commit.
* A mismatch will prevent the commit from succeeding.
* This is essential for operations like S3 put to ensure atomicity.
*/
private boolean atomicKeyCreation;

private volatile boolean closed;
private volatile boolean closing;
// how much of data is actually written yet to underlying stream
Expand Down Expand Up @@ -130,7 +122,6 @@ private ECKeyOutputStream(Builder builder) {
return flushStripeFromQueue();
});
this.flushCheckpoint = new AtomicLong(0);
this.atomicKeyCreation = builder.getAtomicKeyCreation();
}

@Override
Expand Down Expand Up @@ -489,12 +480,6 @@ public void close() throws IOException {
Preconditions.checkArgument(writeOffset == offset,
"Expected writeOffset= " + writeOffset
+ " Expected offset=" + offset);
if (atomicKeyCreation) {
long expectedSize = blockOutputStreamEntryPool.getDataSize();
Preconditions.checkState(expectedSize == offset, String.format(
"Expected: %d and actual %d write sizes do not match",
expectedSize, offset));
}
for (CheckedRunnable<IOException> preCommit : preCommits) {
preCommit.run();
}
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,35 @@
/*
* 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.hadoop.ozone.client.io;

import jakarta.annotation.Nonnull;
import java.io.IOException;
import java.util.List;
import org.apache.hadoop.ozone.om.helpers.OmMultipartCommitUploadPartInfo;
import org.apache.ratis.util.function.CheckedRunnable;

/**
* Common commit-time behavior for key output implementations.
*/
interface KeyCommitOutput extends KeyMetadataAware {

void setPreCommits(
@Nonnull List<CheckedRunnable<IOException>> preCommits);

OmMultipartCommitUploadPartInfo getCommitUploadPartInfo();
}
Original file line number Diff line number Diff line change
Expand Up @@ -60,7 +60,7 @@
* TODO : currently not support multi-thread access.
*/
public class KeyDataStreamOutput extends AbstractDataStreamOutput
implements KeyMetadataAware {
implements KeyCommitOutput {

private static final Logger LOG =
LoggerFactory.getLogger(KeyDataStreamOutput.class);
Expand All @@ -77,16 +77,9 @@ public class KeyDataStreamOutput extends AbstractDataStreamOutput

private long clientID;

/**
* Indicates if an atomic write is required. When set to true,
* the amount of data written must match the declared size during the commit.
* A mismatch will prevent the commit from succeeding.
* This is essential for operations like S3 put to ensure atomicity.
*/
private boolean atomicKeyCreation;

private List<CheckedRunnable<IOException>> preCommits = Collections.emptyList();

@Override
public void setPreCommits(@Nonnull List<CheckedRunnable<IOException>> preCommits) {
this.preCommits = preCommits;
}
Expand Down Expand Up @@ -129,7 +122,6 @@ public KeyDataStreamOutput() {

this.writeOffset = 0;
this.clientID = 0L;
this.atomicKeyCreation = false;
}

@SuppressWarnings({"parameternumber", "squid:S00107"})
Expand All @@ -140,8 +132,7 @@ public KeyDataStreamOutput(
OzoneManagerProtocol omClient, int chunkSize,
String requestId, ReplicationConfig replicationConfig,
String uploadID, int partNumber, boolean isMultipart,
boolean unsafeByteBufferConversion,
boolean atomicKeyCreation
boolean unsafeByteBufferConversion
) {
super(HddsClientUtils.getRetryPolicyByException(
config.getMaxRetryCount(), config.getRetryInterval()));
Expand All @@ -162,7 +153,6 @@ public KeyDataStreamOutput(
// encrypted bucket.
this.writeOffset = 0;
this.clientID = handler.getId();
this.atomicKeyCreation = atomicKeyCreation;
}

/**
Expand Down Expand Up @@ -457,12 +447,6 @@ public void close() throws IOException {
if (!isException()) {
Preconditions.checkArgument(writeOffset == offset);
}
if (atomicKeyCreation) {
long expectedSize = blockDataStreamOutputEntryPool.getDataSize();
Preconditions.checkArgument(expectedSize == offset,
String.format("Expected: %d and actual %d write sizes do not match",
expectedSize, offset));
}
for (CheckedRunnable<IOException> preCommit : preCommits) {
preCommit.run();
}
Expand All @@ -472,6 +456,7 @@ public void close() throws IOException {
}
}

@Override
public OmMultipartCommitUploadPartInfo getCommitUploadPartInfo() {
return blockDataStreamOutputEntryPool.getCommitUploadPartInfo();
}
Expand Down Expand Up @@ -501,7 +486,6 @@ public static class Builder {
private boolean unsafeByteBufferConversion;
private OzoneClientConfig clientConfig;
private ReplicationConfig replicationConfig;
private boolean atomicKeyCreation = false;

public Builder setMultipartUploadID(String uploadID) {
this.multipartUploadID = uploadID;
Expand Down Expand Up @@ -553,11 +537,6 @@ public Builder setReplicationConfig(ReplicationConfig replConfig) {
return this;
}

public Builder setAtomicKeyCreation(boolean atomicKey) {
this.atomicKeyCreation = atomicKey;
return this;
}

public KeyDataStreamOutput build() {
return new KeyDataStreamOutput(
clientConfig,
Expand All @@ -570,8 +549,7 @@ public KeyDataStreamOutput build() {
multipartUploadID,
multipartNumber,
isMultipartKey,
unsafeByteBufferConversion,
atomicKeyCreation);
unsafeByteBufferConversion);
}

}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -76,7 +76,7 @@
* TODO : currently not support multi-thread access.
*/
public class KeyOutputStream extends OutputStream
implements Syncable, KeyMetadataAware {
implements Syncable, KeyCommitOutput {

private static final Logger LOG =
LoggerFactory.getLogger(KeyOutputStream.class);
Expand All @@ -98,13 +98,6 @@ public class KeyOutputStream extends OutputStream
private long clientID;
private StreamBufferArgs streamBufferArgs;

/**
* Indicates if an atomic write is required. When set to true,
* the amount of data written must match the declared size during the commit.
* A mismatch will prevent the commit from succeeding.
* This is essential for operations like S3 put to ensure atomicity.
*/
private boolean atomicKeyCreation;
private ContainerClientMetrics clientMetrics;
private OzoneManagerVersion ozoneManagerVersion;
private final Lock writeLock = new ReentrantLock();
Expand All @@ -114,6 +107,7 @@ public class KeyOutputStream extends OutputStream
private final KeyOutputStreamSemaphore keyOutputStreamSemaphore;
private List<CheckedRunnable<IOException>> preCommits = Collections.emptyList();

@Override
public void setPreCommits(@Nonnull List<CheckedRunnable<IOException>> preCommits) {
this.preCommits = preCommits;
}
Expand Down Expand Up @@ -186,7 +180,6 @@ public KeyOutputStream(Builder b) {
this.isException = false;
this.writeOffset = 0;
this.clientID = b.getOpenHandler().getId();
this.atomicKeyCreation = b.getAtomicKeyCreation();
this.streamBufferArgs = b.getStreamBufferArgs();
this.clientMetrics = b.getClientMetrics();
this.ozoneManagerVersion = b.ozoneManagerVersion;
Expand Down Expand Up @@ -656,12 +649,6 @@ private void closeInternal() throws IOException {
if (!isException) {
Preconditions.checkArgument(writeOffset == offset);
}
if (atomicKeyCreation) {
long expectedSize = blockOutputStreamEntryPool.getDataSize();
Preconditions.checkState(expectedSize == offset,
String.format("Expected: %d and actual %d write sizes do not match",
expectedSize, offset));
}
for (CheckedRunnable<IOException> preCommit : preCommits) {
preCommit.run();
}
Expand All @@ -671,7 +658,8 @@ private void closeInternal() throws IOException {
}
}

synchronized OmMultipartCommitUploadPartInfo
@Override
public synchronized OmMultipartCommitUploadPartInfo
getCommitUploadPartInfo() {
return blockOutputStreamEntryPool.getCommitUploadPartInfo();
}
Expand Down Expand Up @@ -701,7 +689,6 @@ public static class Builder {
private OzoneClientConfig clientConfig;
private ReplicationConfig replicationConfig;
private ContainerClientMetrics clientMetrics;
private boolean atomicKeyCreation = false;
private StreamBufferArgs streamBufferArgs;
private Supplier<ExecutorService> executorServiceSupplier;
private OzoneManagerVersion ozoneManagerVersion;
Expand Down Expand Up @@ -800,11 +787,6 @@ public Builder setReplicationConfig(ReplicationConfig replConfig) {
return this;
}

public Builder setAtomicKeyCreation(boolean atomicKey) {
this.atomicKeyCreation = atomicKey;
return this;
}

public Builder setClientMetrics(ContainerClientMetrics clientMetrics) {
this.clientMetrics = clientMetrics;
return this;
Expand All @@ -814,10 +796,6 @@ public ContainerClientMetrics getClientMetrics() {
return clientMetrics;
}

public boolean getAtomicKeyCreation() {
return atomicKeyCreation;
}

public Builder setExecutorServiceSupplier(Supplier<ExecutorService> executorServiceSupplier) {
this.executorServiceSupplier = executorServiceSupplier;
return this;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,13 +20,15 @@
import java.io.IOException;
import java.io.OutputStream;
import java.nio.ByteBuffer;
import java.util.List;
import java.util.Map;
import java.util.Objects;
import java.util.Optional;
import org.apache.hadoop.crypto.CryptoOutputStream;
import org.apache.hadoop.fs.Syncable;
import org.apache.hadoop.hdds.scm.storage.ByteBufferStreamOutput;
import org.apache.hadoop.ozone.om.helpers.OmMultipartCommitUploadPartInfo;
import org.apache.ratis.util.function.CheckedRunnable;

/**
* OzoneDataStreamOutput is used to write data into Ozone.
Expand Down Expand Up @@ -100,40 +102,67 @@ public synchronized void close() throws IOException {
}

public OmMultipartCommitUploadPartInfo getCommitUploadPartInfo() {
KeyDataStreamOutput keyDataStreamOutput = getKeyDataStreamOutput();
if (keyDataStreamOutput != null) {
return keyDataStreamOutput.getCommitUploadPartInfo();
KeyCommitOutput keyCommitOutput = getKeyCommitOutput();
if (keyCommitOutput != null) {
return keyCommitOutput.getCommitUploadPartInfo();
}
// Otherwise return null.
return null;
}

public KeyDataStreamOutput getKeyDataStreamOutput() {
if (byteBufferStreamOutput instanceof KeyDataStreamOutput) {
return ((KeyDataStreamOutput) byteBufferStreamOutput);
}
if (byteBufferStreamOutput instanceof OzoneOutputStream) {
OutputStream outputStream =
((OzoneOutputStream) byteBufferStreamOutput).getOutputStream();
if (outputStream instanceof KeyDataStreamOutput) {
return ((KeyDataStreamOutput) outputStream);
} else if (outputStream instanceof CryptoOutputStream) {
OutputStream wrappedStream =
((CryptoOutputStream) outputStream).getWrappedStream();
if (wrappedStream instanceof KeyDataStreamOutput) {
return ((KeyDataStreamOutput) wrappedStream);
}
} else if (outputStream instanceof CipherOutputStreamOzone) {
OutputStream wrappedStream =
((CipherOutputStreamOzone) outputStream).getWrappedStream();
if (wrappedStream instanceof KeyDataStreamOutput) {
return ((KeyDataStreamOutput) wrappedStream);
}
OutputStream unwrappedStream = unwrap(outputStream);
if (unwrappedStream instanceof KeyDataStreamOutput) {
return ((KeyDataStreamOutput) unwrappedStream);
}
}
// Otherwise return null.
return null;
}

private KeyCommitOutput getKeyCommitOutput() {
if (byteBufferStreamOutput instanceof KeyCommitOutput) {
return (KeyCommitOutput) byteBufferStreamOutput;
}
if (byteBufferStreamOutput instanceof OzoneOutputStream) {
OutputStream outputStream =
((OzoneOutputStream) byteBufferStreamOutput).getOutputStream();
OutputStream unwrappedStream = unwrap(outputStream);
if (unwrappedStream instanceof KeyCommitOutput) {
return (KeyCommitOutput) unwrappedStream;
}
} else if (byteBufferStreamOutput instanceof KeyDataStreamOutput) {
return ((KeyDataStreamOutput) byteBufferStreamOutput);
}
// Otherwise return null.
return null;
}

public void setPreCommits(
List<CheckedRunnable<IOException>> preCommits) {
KeyCommitOutput keyCommitOutput = getKeyCommitOutput();
if (keyCommitOutput != null) {
keyCommitOutput.setPreCommits(preCommits);
return;
}
throw new IllegalStateException(
"Output stream is not backed by KeyCommitOutput: " +
byteBufferStreamOutput.getClass());
}

private static OutputStream unwrap(OutputStream outputStream) {
if (outputStream instanceof CryptoOutputStream) {
return ((CryptoOutputStream) outputStream).getWrappedStream();
} else if (outputStream instanceof CipherOutputStreamOzone) {
return ((CipherOutputStreamOzone) outputStream).getWrappedStream();
}
return outputStream;
}

@Override
public void hflush() throws IOException {
hsync();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1224,8 +1224,6 @@ OzoneKey headObject(String volumeName, String bucketName,
*/
void setThreadLocalS3Auth(S3Auth s3Auth);

void setIsS3Request(boolean isS3Request);

/**
* Gets the S3 Authentication information that is attached to the thread.
* @return S3 Authentication information.
Expand Down
Loading
Loading