steveloughran commented on code in PR #7396: URL: https://github.com/apache/hadoop/pull/7396#discussion_r1959886998
########## hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/files/Etag.java: ########## @@ -0,0 +1,135 @@ +/* + * 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.fs.s3a.commit.files; + +import java.io.Serializable; +import java.util.StringJoiner; + +import software.amazon.awssdk.services.s3.model.ChecksumAlgorithm; +import software.amazon.awssdk.services.s3.model.CompletedPart; + +/** + * Stores ETag and checksum values from {@link CompletedPart} responses from S3. + * These values need to be stored to be later passed to the + * {@link software.amazon.awssdk.services.s3.model.CompleteMultipartUploadRequest + * CompleteMultipartUploadRequest} + */ +public class Etag implements Serializable { Review Comment: overused; needs a name like `UploadEtag` ########## hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/troubleshooting_s3a.md: ########## @@ -387,6 +387,28 @@ Happens if a multipart upload is being completed, but one of the parts is missin * A magic committer job's list of in-progress uploads somehow got corrupted * Bug in the S3A codebase (rare, but not impossible...) +### <a name="object_lock_parameters"></a> Status Code 400 "Content-MD5 OR x-amz-checksum- HTTP header is required for Put Object requests with Object Lock parameters" +``` +software.amazon.awssdk.services.s3.model.S3Exception: Content-MD5 OR x-amz-checksum- HTTP header is required for Put Object requests with Object Lock parameters (Service: S3, Status Code: 400, Request ID: 1122334455, Extended Request ID: ...): Review Comment: always good to get extra troubleshooting sections -appreciated ########## hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java: ########## @@ -1743,4 +1753,26 @@ static void maybeIsolateClassloader(Configuration conf, ClassLoader classLoader) } } + /** + * Get the checksum algorithm to be used for data integrity check of the objects in S3. + * This operation includes validating if the provided value is a supported checksum algorithm. + * @param conf configuration to scan + * @return the checksum algorithm to be passed on S3 requests + * @throws IllegalArgumentException if the checksum algorithm is not known or not supported + */ + public static ChecksumAlgorithm getChecksumAlgorithm(Configuration conf) { + final String checksumAlgorithmString = conf.get(CHECKSUM_ALGORITHM); Review Comment: getTrimmed() ########## hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestS3AMultipartUploaderSupport.java: ########## @@ -41,35 +44,52 @@ public class TestS3AMultipartUploaderSupport extends HadoopTestBase { @Test public void testRoundTrip() throws Throwable { - PartHandlePayload result = roundTrip(999, "tag", 1); + PartHandlePayload result = roundTrip(999, "tag", 1, null, null); assertEquals(PATH, result.getPath()); assertEquals(UPLOAD, result.getUploadId()); assertEquals(999, result.getPartNumber()); assertEquals("tag", result.getEtag()); assertEquals(1, result.getLen()); + assertNull(result.getChecksumAlgorithm()); + assertNull(result.getChecksum()); } @Test public void testRoundTrip2() throws Throwable { long len = 1L + Integer.MAX_VALUE; PartHandlePayload result = - roundTrip(1, "11223344", len); + roundTrip(1, "11223344", len, null, null); assertEquals(1, result.getPartNumber()); assertEquals("11223344", result.getEtag()); assertEquals(len, result.getLen()); + assertNull(result.getChecksumAlgorithm()); + assertNull(result.getChecksum()); + } + + @Test + public void testRoundTripWithChecksum() throws Throwable { Review Comment: again, assertJ. yes, the class will be a mix, but that's OK ########## hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3ADeleteOnExit.java: ########## @@ -89,6 +90,17 @@ public void testDeleteOnExit() throws Exception { assertEquals(0, testFs.getDeleteOnDnExitCount()); } + @Test + public void testCreateRequestFactoryWithInvalidChecksumAlgorithm() throws Exception { + Configuration conf = createConfiguration(); + conf.set(Constants.CHECKSUM_ALGORITHM, "INVALID"); + TestS3AFileSystem testFs = new TestS3AFileSystem(); + URI uri = URI.create(FS_S3A + "://" + BUCKET); + final IllegalArgumentException exception = intercept(IllegalArgumentException.class, + () -> testFs.initialize(uri, conf)); + assertEquals("Checksum algorithm is not supported: INVALID", exception.getMessage()); Review Comment: use AssertJ assertions in new tests...better assertions and makes the move to Junit5 easier ########## hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestS3AMultipartUploaderSupport.java: ########## @@ -41,35 +44,52 @@ public class TestS3AMultipartUploaderSupport extends HadoopTestBase { @Test public void testRoundTrip() throws Throwable { - PartHandlePayload result = roundTrip(999, "tag", 1); + PartHandlePayload result = roundTrip(999, "tag", 1, null, null); assertEquals(PATH, result.getPath()); assertEquals(UPLOAD, result.getUploadId()); assertEquals(999, result.getPartNumber()); assertEquals("tag", result.getEtag()); assertEquals(1, result.getLen()); + assertNull(result.getChecksumAlgorithm()); Review Comment: add as assertJ with descriptions of what is being checked ########## hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java: ########## @@ -1743,4 +1753,26 @@ static void maybeIsolateClassloader(Configuration conf, ClassLoader classLoader) } } + /** + * Get the checksum algorithm to be used for data integrity check of the objects in S3. + * This operation includes validating if the provided value is a supported checksum algorithm. + * @param conf configuration to scan + * @return the checksum algorithm to be passed on S3 requests + * @throws IllegalArgumentException if the checksum algorithm is not known or not supported + */ + public static ChecksumAlgorithm getChecksumAlgorithm(Configuration conf) { + final String checksumAlgorithmString = conf.get(CHECKSUM_ALGORITHM); + if (StringUtils.isBlank(checksumAlgorithmString)) { + return null; + } + final ChecksumAlgorithm checksumAlgorithm = + ChecksumAlgorithm.fromValue(checksumAlgorithmString); + if (!SUPPORTED_CHECKSUM_ALGORITHMS.contains(checksumAlgorithm)) { Review Comment: prefer Precondition.checkState() ########## hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java: ########## @@ -135,6 +139,12 @@ public final class S3AUtils { private static final String BUCKET_PATTERN = FS_S3A_BUCKET_PREFIX + "%s.%s"; + public static final Set<ChecksumAlgorithm> SUPPORTED_CHECKSUM_ALGORITHMS = ImmutableSet.of( Review Comment: let's move this into its own class `...s3a.impl.ChecksumSupport`. S3AUtils is too big and a source of cherrypick pain, so we prefer isolating things now -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org --------------------------------------------------------------------- To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org For additional commands, e-mail: common-issues-h...@hadoop.apache.org