Skip to content

Commit c1bcdea

Browse files
authored
HDDS-10832. Client should switch to streaming based on OpenKeySession replication (apache#6683)
1 parent 8a23991 commit c1bcdea

2 files changed

Lines changed: 59 additions & 50 deletions

File tree

hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java

Lines changed: 50 additions & 40 deletions
Original file line numberDiff line numberDiff line change
@@ -50,6 +50,7 @@
5050
import org.apache.hadoop.hdds.scm.XceiverClientFactory;
5151
import org.apache.hadoop.hdds.scm.XceiverClientManager;
5252
import org.apache.hadoop.hdds.scm.client.ClientTrustManager;
53+
import org.apache.hadoop.hdds.scm.storage.ByteBufferStreamOutput;
5354
import org.apache.hadoop.hdds.security.x509.certificate.client.CACertificateProvider;
5455
import org.apache.hadoop.hdds.scm.client.HddsClientUtils;
5556
import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
@@ -1959,11 +1960,16 @@ public OzoneOutputStream createMultipartKey(
19591960
long size, int partNumber, String uploadID) throws IOException {
19601961
final OpenKeySession openKey = newMultipartOpenKey(
19611962
volumeName, bucketName, keyName, size, partNumber, uploadID, false);
1963+
return createMultipartOutputStream(openKey, uploadID, partNumber);
1964+
}
1965+
1966+
private OzoneOutputStream createMultipartOutputStream(
1967+
OpenKeySession openKey, String uploadID, int partNumber
1968+
) throws IOException {
19621969
KeyOutputStream keyOutputStream = createKeyOutputStream(openKey)
19631970
.setMultipartNumber(partNumber)
19641971
.setMultipartUploadID(uploadID)
19651972
.setIsMultipartKey(true)
1966-
.setAtomicKeyCreation(isS3GRequest.get())
19671973
.build();
19681974
return createOutputStream(openKey, keyOutputStream);
19691975
}
@@ -1979,29 +1985,25 @@ public OzoneDataStreamOutput createMultipartStreamKey(
19791985
throws IOException {
19801986
final OpenKeySession openKey = newMultipartOpenKey(
19811987
volumeName, bucketName, keyName, size, partNumber, uploadID, true);
1982-
// Amazon S3 never adds partial objects, So for S3 requests we need to
1983-
// set atomicKeyCreation to true
1984-
// refer: https://docs.aws.amazon.com/AmazonS3/latest/API/API_PutObject.html
1985-
KeyDataStreamOutput keyOutputStream =
1986-
new KeyDataStreamOutput.Builder()
1987-
.setHandler(openKey)
1988-
.setXceiverClientManager(xceiverClientManager)
1989-
.setOmClient(ozoneManagerClient)
1990-
.setReplicationConfig(openKey.getKeyInfo().getReplicationConfig())
1991-
.setMultipartNumber(partNumber)
1992-
.setMultipartUploadID(uploadID)
1993-
.setIsMultipartKey(true)
1994-
.enableUnsafeByteBufferConversion(unsafeByteBufferConversion)
1995-
.setConfig(clientConfig)
1996-
.setAtomicKeyCreation(isS3GRequest.get())
1997-
.build();
1998-
keyOutputStream
1999-
.addPreallocateBlocks(
2000-
openKey.getKeyInfo().getLatestVersionLocations(),
2001-
openKey.getOpenVersion());
2002-
final OzoneOutputStream out = createSecureOutputStream(
2003-
openKey, keyOutputStream, null);
2004-
return new OzoneDataStreamOutput(out != null ? out : keyOutputStream);
1988+
final ByteBufferStreamOutput out;
1989+
ReplicationConfig replicationConfig = openKey.getKeyInfo().getReplicationConfig();
1990+
if (replicationConfig.getReplicationType() == HddsProtos.ReplicationType.RATIS) {
1991+
KeyDataStreamOutput keyOutputStream = newKeyOutputStreamBuilder()
1992+
.setHandler(openKey)
1993+
.setReplicationConfig(replicationConfig)
1994+
.setMultipartNumber(partNumber)
1995+
.setMultipartUploadID(uploadID)
1996+
.setIsMultipartKey(true)
1997+
.build();
1998+
keyOutputStream.addPreallocateBlocks(
1999+
openKey.getKeyInfo().getLatestVersionLocations(),
2000+
openKey.getOpenVersion());
2001+
final OzoneOutputStream secureOut = createSecureOutputStream(openKey, keyOutputStream, null);
2002+
out = secureOut != null ? secureOut : keyOutputStream;
2003+
} else {
2004+
out = createMultipartOutputStream(openKey, uploadID, partNumber);
2005+
}
2006+
return new OzoneDataStreamOutput(out);
20052007
}
20062008

20072009
@Override
@@ -2403,25 +2405,33 @@ private OzoneDataStreamOutput createDataStreamOutput(OpenKeySession openKey)
24032405
throws IOException {
24042406
final ReplicationConfig replicationConfig
24052407
= openKey.getKeyInfo().getReplicationConfig();
2408+
final ByteBufferStreamOutput out;
2409+
if (replicationConfig.getReplicationType() == HddsProtos.ReplicationType.RATIS) {
2410+
KeyDataStreamOutput keyOutputStream = newKeyOutputStreamBuilder()
2411+
.setHandler(openKey)
2412+
.setReplicationConfig(replicationConfig)
2413+
.build();
2414+
keyOutputStream.addPreallocateBlocks(
2415+
openKey.getKeyInfo().getLatestVersionLocations(),
2416+
openKey.getOpenVersion());
2417+
final OzoneOutputStream secureOut = createSecureOutputStream(openKey, keyOutputStream, null);
2418+
out = secureOut != null ? secureOut : keyOutputStream;
2419+
} else {
2420+
out = createOutputStream(openKey);
2421+
}
2422+
return new OzoneDataStreamOutput(out);
2423+
}
2424+
2425+
private KeyDataStreamOutput.Builder newKeyOutputStreamBuilder() {
24062426
// Amazon S3 never adds partial objects, So for S3 requests we need to
24072427
// set atomicKeyCreation to true
24082428
// refer: https://docs.aws.amazon.com/AmazonS3/latest/API/API_PutObject.html
2409-
KeyDataStreamOutput keyOutputStream =
2410-
new KeyDataStreamOutput.Builder()
2411-
.setHandler(openKey)
2412-
.setXceiverClientManager(xceiverClientManager)
2413-
.setOmClient(ozoneManagerClient)
2414-
.setReplicationConfig(replicationConfig)
2415-
.enableUnsafeByteBufferConversion(unsafeByteBufferConversion)
2416-
.setConfig(clientConfig)
2417-
.setAtomicKeyCreation(isS3GRequest.get())
2418-
.build();
2419-
keyOutputStream
2420-
.addPreallocateBlocks(openKey.getKeyInfo().getLatestVersionLocations(),
2421-
openKey.getOpenVersion());
2422-
final OzoneOutputStream out = createSecureOutputStream(
2423-
openKey, keyOutputStream, null);
2424-
return new OzoneDataStreamOutput(out != null ? out : keyOutputStream);
2429+
return new KeyDataStreamOutput.Builder()
2430+
.setXceiverClientManager(xceiverClientManager)
2431+
.setOmClient(ozoneManagerClient)
2432+
.enableUnsafeByteBufferConversion(unsafeByteBufferConversion)
2433+
.setConfig(clientConfig)
2434+
.setAtomicKeyCreation(isS3GRequest.get());
24252435
}
24262436

24272437
private OzoneOutputStream createOutputStream(OpenKeySession openKey)

hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneRpcClientWithRatis.java

Lines changed: 9 additions & 10 deletions
Original file line numberDiff line numberDiff line change
@@ -27,6 +27,7 @@
2727
import java.security.MessageDigest;
2828
import java.security.NoSuchAlgorithmException;
2929
import java.util.HashMap;
30+
import java.util.Locale;
3031
import java.util.UUID;
3132
import java.util.concurrent.CountDownLatch;
3233
import java.util.concurrent.ThreadLocalRandom;
@@ -61,6 +62,8 @@
6162
import org.junit.jupiter.api.AfterAll;
6263
import org.junit.jupiter.api.BeforeAll;
6364
import org.junit.jupiter.api.Test;
65+
import org.junit.jupiter.params.ParameterizedTest;
66+
import org.junit.jupiter.params.provider.MethodSource;
6467

6568
import static java.nio.charset.StandardCharsets.UTF_8;
6669
import static org.apache.hadoop.hdds.client.ReplicationFactor.ONE;
@@ -170,12 +173,13 @@ void testGetKeyAndFileWithNetworkTopology() throws IOException {
170173
}
171174
}
172175

173-
@Test
174-
public void testMultiPartUploadWithStream()
176+
@ParameterizedTest
177+
@MethodSource("replicationConfigs")
178+
void testMultiPartUploadWithStream(ReplicationConfig replicationConfig)
175179
throws IOException, NoSuchAlgorithmException {
176180
String volumeName = UUID.randomUUID().toString();
177-
String bucketName = UUID.randomUUID().toString();
178-
String keyName = UUID.randomUUID().toString();
181+
String bucketName = replicationConfig.getReplicationType().name().toLowerCase(Locale.ROOT) + "-bucket";
182+
String keyName = replicationConfig.getReplication();
179183

180184
byte[] sampleData = new byte[1024 * 8];
181185

@@ -186,11 +190,6 @@ public void testMultiPartUploadWithStream()
186190
volume.createBucket(bucketName);
187191
OzoneBucket bucket = volume.getBucket(bucketName);
188192

189-
ReplicationConfig replicationConfig =
190-
ReplicationConfig.fromTypeAndFactor(
191-
ReplicationType.RATIS,
192-
THREE);
193-
194193
OmMultipartInfo multipartInfo = bucket.initiateMultipartUpload(keyName,
195194
replicationConfig);
196195

@@ -210,7 +209,7 @@ public void testMultiPartUploadWithStream()
210209
OzoneMultipartUploadPartListParts parts =
211210
bucket.listParts(keyName, uploadID, 0, 1);
212211

213-
assertEquals(parts.getPartInfoList().size(), 1);
212+
assertEquals(1, parts.getPartInfoList().size());
214213

215214
OzoneMultipartUploadPartListParts.PartInfo partInfo =
216215
parts.getPartInfoList().get(0);

0 commit comments

Comments
 (0)