Skip to content

Commit 37eead5

Browse files
committed
HADOOP-18708: Support S3 Client Side Encryption(CSE) With AWS SDK V2
1 parent dc56fc3 commit 37eead5

File tree

50 files changed

+2209
-174
lines changed

Some content is hidden

Large Commits have some content hidden by default. Use the searchbox below for content that may be hidden.

50 files changed

+2209
-174
lines changed

hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/MultipartUploader.java

Lines changed: 3 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -57,6 +57,7 @@ CompletableFuture<UploadHandle> startUpload(Path filePath)
5757
* It is possible to have parts uploaded in any order (or in parallel).
5858
* @param uploadId Identifier from {@link #startUpload(Path)}.
5959
* @param partNumber Index of the part relative to others.
60+
* @param isLastPart is the part the last part of the upload?
6061
* @param filePath Target path for upload (as {@link #startUpload(Path)}).
6162
* @param inputStream Data for this part. Implementations MUST close this
6263
* stream after reading in the data.
@@ -67,6 +68,7 @@ CompletableFuture<UploadHandle> startUpload(Path filePath)
6768
CompletableFuture<PartHandle> putPart(
6869
UploadHandle uploadId,
6970
int partNumber,
71+
boolean isLastPart,
7072
Path filePath,
7173
InputStream inputStream,
7274
long lengthInBytes)
@@ -77,7 +79,7 @@ CompletableFuture<PartHandle> putPart(
7779
* @param uploadId Identifier from {@link #startUpload(Path)}.
7880
* @param filePath Target path for upload (as {@link #startUpload(Path)}.
7981
* @param handles non-empty map of part number to part handle.
80-
* from {@link #putPart(UploadHandle, int, Path, InputStream, long)}.
82+
* from {@link #putPart(UploadHandle, int, boolean, Path, InputStream, long)}.
8183
* @return unique PathHandle identifier for the uploaded file.
8284
* @throws IOException IO failure
8385
*/

hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/AbstractMultipartUploader.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -101,7 +101,7 @@ protected void checkPartHandles(Map<Integer, PartHandle> partHandles) {
101101

102102
/**
103103
* Check all the arguments to the
104-
* {@link MultipartUploader#putPart(UploadHandle, int, Path, InputStream, long)}
104+
* {@link MultipartUploader#putPart(UploadHandle, int, boolean, Path, InputStream, long)}
105105
* operation.
106106
* @param filePath Target path for upload (as {@link #startUpload(Path)}).
107107
* @param inputStream Data for this part. Implementations MUST close this

hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/FileSystemMultipartUploader.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -111,7 +111,7 @@ public CompletableFuture<UploadHandle> startUpload(Path filePath)
111111

112112
@Override
113113
public CompletableFuture<PartHandle> putPart(UploadHandle uploadId,
114-
int partNumber, Path filePath,
114+
int partNumber, boolean isLastPart, Path filePath,
115115
InputStream inputStream,
116116
long lengthInBytes)
117117
throws IOException {

hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractMultipartUploaderTest.java

Lines changed: 21 additions & 15 deletions
Original file line numberDiff line numberDiff line change
@@ -249,7 +249,7 @@ public void testSingleUpload() throws Exception {
249249
// was interpreted as an inconsistent write.
250250
MultipartUploader completer = uploader0;
251251
// and upload with uploader 1 to validate cross-uploader uploads
252-
PartHandle partHandle = putPart(file, uploadHandle, 1, payload);
252+
PartHandle partHandle = putPart(file, uploadHandle, 1, true, payload);
253253
partHandles.put(1, partHandle);
254254
PathHandle fd = complete(completer, uploadHandle, file,
255255
partHandles);
@@ -317,12 +317,13 @@ protected PartHandle buildAndPutPart(
317317
final Path file,
318318
final UploadHandle uploadHandle,
319319
final int index,
320+
final boolean isLastPart,
320321
final MessageDigest origDigest) throws IOException {
321322
byte[] payload = generatePayload(index);
322323
if (origDigest != null) {
323324
origDigest.update(payload);
324325
}
325-
return putPart(file, uploadHandle, index, payload);
326+
return putPart(file, uploadHandle, index, isLastPart, payload);
326327
}
327328

328329
/**
@@ -331,13 +332,15 @@ protected PartHandle buildAndPutPart(
331332
* @param file destination
332333
* @param uploadHandle handle
333334
* @param index index of part
335+
* @param isLastPart is last part of the upload ?
334336
* @param payload byte array of payload
335337
* @return the part handle
336338
* @throws IOException IO failure.
337339
*/
338340
protected PartHandle putPart(final Path file,
339341
final UploadHandle uploadHandle,
340342
final int index,
343+
final boolean isLastPart,
341344
final byte[] payload) throws IOException {
342345
ContractTestUtils.NanoTimer timer = new ContractTestUtils.NanoTimer();
343346
PartHandle partHandle;
@@ -347,7 +350,7 @@ protected PartHandle putPart(final Path file,
347350
payload.length,
348351
file)) {
349352
partHandle = awaitFuture(getUploader(index)
350-
.putPart(uploadHandle, index, file,
353+
.putPart(uploadHandle, index, isLastPart, file,
351354
new ByteArrayInputStream(payload),
352355
payload.length));
353356
}
@@ -488,7 +491,7 @@ public void testMultipartUpload() throws Exception {
488491
MessageDigest origDigest = DigestUtils.getMd5Digest();
489492
int payloadCount = getTestPayloadCount();
490493
for (int i = 1; i <= payloadCount; ++i) {
491-
PartHandle partHandle = buildAndPutPart(file, uploadHandle, i,
494+
PartHandle partHandle = buildAndPutPart(file, uploadHandle, i, i == payloadCount,
492495
origDigest);
493496
partHandles.put(i, partHandle);
494497
}
@@ -515,7 +518,7 @@ public void testMultipartUploadEmptyPart() throws Exception {
515518
origDigest.update(payload);
516519
InputStream is = new ByteArrayInputStream(payload);
517520
PartHandle partHandle = awaitFuture(
518-
uploader.putPart(uploadHandle, 1, file, is, payload.length));
521+
uploader.putPart(uploadHandle, 1, true, file, is, payload.length));
519522
partHandles.put(1, partHandle);
520523
completeUpload(file, uploadHandle, partHandles, origDigest, 0);
521524
}
@@ -530,7 +533,7 @@ public void testUploadEmptyBlock() throws Exception {
530533
Path file = methodPath();
531534
UploadHandle uploadHandle = startUpload(file);
532535
Map<Integer, PartHandle> partHandles = new HashMap<>();
533-
partHandles.put(1, putPart(file, uploadHandle, 1, new byte[0]));
536+
partHandles.put(1, putPart(file, uploadHandle, 1, true, new byte[0]));
534537
completeUpload(file, uploadHandle, partHandles, null, 0);
535538
}
536539

@@ -550,7 +553,8 @@ public void testMultipartUploadReverseOrder() throws Exception {
550553
origDigest.update(payload);
551554
}
552555
for (int i = payloadCount; i > 0; --i) {
553-
partHandles.put(i, buildAndPutPart(file, uploadHandle, i, null));
556+
partHandles.put(i, buildAndPutPart(file, uploadHandle, i, i == payloadCount,
557+
null));
554558
}
555559
completeUpload(file, uploadHandle, partHandles, origDigest,
556560
payloadCount * partSizeInBytes());
@@ -574,7 +578,8 @@ public void testMultipartUploadReverseOrderNonContiguousPartNumbers()
574578
}
575579
Map<Integer, PartHandle> partHandles = new HashMap<>();
576580
for (int i = payloadCount; i > 0; i -= 2) {
577-
partHandles.put(i, buildAndPutPart(file, uploadHandle, i, null));
581+
partHandles.put(i, buildAndPutPart(file, uploadHandle, i, i == payloadCount,
582+
null));
578583
}
579584
completeUpload(file, uploadHandle, partHandles, origDigest,
580585
getTestPayloadCount() * partSizeInBytes());
@@ -591,7 +596,7 @@ public void testMultipartUploadAbort() throws Exception {
591596
UploadHandle uploadHandle = startUpload(file);
592597
Map<Integer, PartHandle> partHandles = new HashMap<>();
593598
for (int i = 12; i > 10; i--) {
594-
partHandles.put(i, buildAndPutPart(file, uploadHandle, i, null));
599+
partHandles.put(i, buildAndPutPart(file, uploadHandle, i, i == 12, null));
595600
}
596601
abortUpload(uploadHandle, file);
597602

@@ -601,7 +606,7 @@ public void testMultipartUploadAbort() throws Exception {
601606

602607
intercept(IOException.class,
603608
() -> awaitFuture(
604-
uploader0.putPart(uploadHandle, 49, file, is, len)));
609+
uploader0.putPart(uploadHandle, 49, true, file, is, len)));
605610
intercept(IOException.class,
606611
() -> complete(uploader0, uploadHandle, file, partHandles));
607612

@@ -701,7 +706,8 @@ public void testPutPartEmptyUploadID() throws Exception {
701706
byte[] payload = generatePayload(1);
702707
InputStream is = new ByteArrayInputStream(payload);
703708
intercept(IllegalArgumentException.class,
704-
() -> uploader0.putPart(emptyHandle, 1, dest, is, payload.length));
709+
() -> uploader0.putPart(emptyHandle, 1, true, dest, is,
710+
payload.length));
705711
}
706712

707713
/**
@@ -715,7 +721,7 @@ public void testCompleteEmptyUploadID() throws Exception {
715721
UploadHandle emptyHandle =
716722
BBUploadHandle.from(ByteBuffer.wrap(new byte[0]));
717723
Map<Integer, PartHandle> partHandles = new HashMap<>();
718-
PartHandle partHandle = putPart(dest, realHandle, 1,
724+
PartHandle partHandle = putPart(dest, realHandle, 1, true,
719725
generatePayload(1, SMALL_FILE));
720726
partHandles.put(1, partHandle);
721727

@@ -743,7 +749,7 @@ public void testDirectoryInTheWay() throws Exception {
743749
UploadHandle uploadHandle = startUpload(file);
744750
Map<Integer, PartHandle> partHandles = new HashMap<>();
745751
int size = SMALL_FILE;
746-
PartHandle partHandle = putPart(file, uploadHandle, 1,
752+
PartHandle partHandle = putPart(file, uploadHandle, 1, true,
747753
generatePayload(1, size));
748754
partHandles.put(1, partHandle);
749755

@@ -802,10 +808,10 @@ public void testConcurrentUploads() throws Throwable {
802808
assertNotEquals("Upload handles match", upload1, upload2);
803809

804810
// put part 1
805-
partHandles1.put(partId1, putPart(file, upload1, partId1, payload1));
811+
partHandles1.put(partId1, putPart(file, upload1, partId1, false, payload1));
806812

807813
// put part2
808-
partHandles2.put(partId2, putPart(file, upload2, partId2, payload2));
814+
partHandles2.put(partId2, putPart(file, upload2, partId2, true, payload2));
809815

810816
// complete part u1. expect its size and digest to
811817
// be as expected.

hadoop-project/pom.xml

Lines changed: 12 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -194,6 +194,7 @@
194194
<surefire.fork.timeout>900</surefire.fork.timeout>
195195
<aws-java-sdk.version>1.12.720</aws-java-sdk.version>
196196
<aws-java-sdk-v2.version>2.25.53</aws-java-sdk-v2.version>
197+
<amazon-s3-encryption-client-java.version>3.1.1</amazon-s3-encryption-client-java.version>
197198
<aws.eventstream.version>1.0.1</aws.eventstream.version>
198199
<hsqldb.version>2.7.1</hsqldb.version>
199200
<frontend-maven-plugin.version>1.11.2</frontend-maven-plugin.version>
@@ -1169,6 +1170,17 @@
11691170
</exclusion>
11701171
</exclusions>
11711172
</dependency>
1173+
<dependency>
1174+
<groupId>software.amazon.encryption.s3</groupId>
1175+
<artifactId>amazon-s3-encryption-client-java</artifactId>
1176+
<version>${amazon-s3-encryption-client-java.version}</version>
1177+
<exclusions>
1178+
<exclusion>
1179+
<groupId>*</groupId>
1180+
<artifactId>*</artifactId>
1181+
</exclusion>
1182+
</exclusions>
1183+
</dependency>
11721184
<dependency>
11731185
<groupId>org.apache.mina</groupId>
11741186
<artifactId>mina-core</artifactId>

hadoop-tools/hadoop-aws/pom.xml

Lines changed: 15 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -466,6 +466,16 @@
466466
<bannedImport>org.apache.hadoop.mapred.**</bannedImport>
467467
</bannedImports>
468468
</restrictImports>
469+
<restrictImports>
470+
<includeTestCode>false</includeTestCode>
471+
<reason>Restrict encryption client imports to encryption client factory</reason>
472+
<exclusions>
473+
<exclusion>org.apache.hadoop.fs.s3a.impl.EncryptionS3ClientFactory</exclusion>
474+
</exclusions>
475+
<bannedImports>
476+
<bannedImport>software.amazon.encryption.s3.**</bannedImport>
477+
</bannedImports>
478+
</restrictImports>
469479
</rules>
470480
</configuration>
471481
</execution>
@@ -510,6 +520,11 @@
510520
<artifactId>bundle</artifactId>
511521
<scope>compile</scope>
512522
</dependency>
523+
<dependency>
524+
<groupId>software.amazon.encryption.s3</groupId>
525+
<artifactId>amazon-s3-encryption-client-java</artifactId>
526+
<scope>provided</scope>
527+
</dependency>
513528
<dependency>
514529
<groupId>org.assertj</groupId>
515530
<artifactId>assertj-core</artifactId>

hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java

Lines changed: 34 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -766,6 +766,40 @@ private Constants() {
766766
public static final String S3_ENCRYPTION_CONTEXT =
767767
"fs.s3a.encryption.context";
768768

769+
/**
770+
* Client side encryption (CSE-CUSTOM) with custom cryptographic material manager class name.
771+
* Custom keyring class name for CSE-KMS.
772+
* value:{@value}
773+
*/
774+
public static final String S3_ENCRYPTION_CSE_CUSTOM_KEYRING_CLASS_NAME =
775+
"fs.s3a.encryption.cse.custom.keyring.class.name";
776+
777+
/**
778+
* Config to provide backward compatibility with V1 encryption client.
779+
* Enabling this configuration will invoke the followings
780+
* 1. Unencrypted s3 objects will be read using unecrypted/base s3 client when CSE is enabled.
781+
* 2. Size of encrypted object will be calculated using ranged S3 calls.
782+
* 3. While listing s3 objects, encryption metadata file with suffix
783+
* {@link #S3_ENCRYPTION_CSE_INSTRUCTION_FILE_SUFFIX} will be skipped.
784+
* This is to provide backward compatibility with V1 client.
785+
* value:{@value}
786+
*/
787+
public static final String S3_ENCRYPTION_CSE_V1_COMPATIBILITY_ENABLED =
788+
"fs.s3a.encryption.cse.v1.compatibility.enabled";
789+
790+
/**
791+
* Default value : {@value}.
792+
*/
793+
public static final boolean S3_ENCRYPTION_CSE_V1_COMPATIBILITY_ENABLED_DEFAULT = false;
794+
795+
/**
796+
* Suffix of instruction file : {@value}.
797+
*/
798+
public static final String S3_ENCRYPTION_CSE_INSTRUCTION_FILE_SUFFIX = ".instruction";
799+
800+
801+
802+
769803
/**
770804
* List of custom Signers. The signer class will be loaded, and the signer
771805
* name will be associated with this signer class in the S3 SDK.

hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/DefaultS3ClientFactory.java

Lines changed: 13 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -41,6 +41,7 @@
4141
import software.amazon.awssdk.regions.Region;
4242
import software.amazon.awssdk.s3accessgrants.plugin.S3AccessGrantsPlugin;
4343
import software.amazon.awssdk.services.s3.S3AsyncClient;
44+
import software.amazon.awssdk.services.s3.S3AsyncClientBuilder;
4445
import software.amazon.awssdk.services.s3.S3BaseClientBuilder;
4546
import software.amazon.awssdk.services.s3.S3Client;
4647
import software.amazon.awssdk.services.s3.S3Configuration;
@@ -160,11 +161,17 @@ public S3AsyncClient createS3AsyncClient(
160161
.thresholdInBytes(parameters.getMultiPartThreshold())
161162
.build();
162163

163-
return configureClientBuilder(S3AsyncClient.builder(), parameters, conf, bucket)
164-
.httpClientBuilder(httpClientBuilder)
165-
.multipartConfiguration(multipartConfiguration)
166-
.multipartEnabled(parameters.isMultipartCopy())
167-
.build();
164+
S3AsyncClientBuilder s3AsyncClientBuilder =
165+
configureClientBuilder(S3AsyncClient.builder(), parameters, conf, bucket)
166+
.httpClientBuilder(httpClientBuilder);
167+
168+
// TODO: Enable multi part upload with cse once it is available.
169+
if (!parameters.isClientSideEncryptionEnabled()) {
170+
s3AsyncClientBuilder.multipartConfiguration(multipartConfiguration)
171+
.multipartEnabled(parameters.isMultipartCopy());
172+
}
173+
174+
return s3AsyncClientBuilder.build();
168175
}
169176

170177
@Override
@@ -373,7 +380,7 @@ private <BuilderT extends S3BaseClientBuilder<BuilderT, ClientT>, ClientT> void
373380
* @param conf config to build the URI from.
374381
* @return an endpoint uri
375382
*/
376-
private static URI getS3Endpoint(String endpoint, final Configuration conf) {
383+
public static URI getS3Endpoint(String endpoint, final Configuration conf) {
377384

378385
boolean secureConnections = conf.getBoolean(SECURE_CONNECTIONS, DEFAULT_SECURE_CONNECTIONS);
379386

0 commit comments

Comments
 (0)