Skip to content

Commit

Permalink
Support CSE-KMS S3 object file size detection via tail read
Browse files Browse the repository at this point in the history
  • Loading branch information
pettyjamesm authored and dain committed Sep 12, 2020
1 parent 551c599 commit 75f426e
Showing 1 changed file with 17 additions and 2 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -35,6 +35,7 @@
import com.amazonaws.services.s3.AmazonS3;
import com.amazonaws.services.s3.AmazonS3Builder;
import com.amazonaws.services.s3.AmazonS3Client;
import com.amazonaws.services.s3.AmazonS3Encryption;
import com.amazonaws.services.s3.AmazonS3EncryptionClient;
import com.amazonaws.services.s3.Headers;
import com.amazonaws.services.s3.internal.Constants;
Expand Down Expand Up @@ -67,6 +68,7 @@
import io.airlift.log.Logger;
import io.airlift.units.DataSize;
import io.airlift.units.Duration;
import io.prestosql.plugin.hive.util.FSDataInputStreamTail;
import org.apache.hadoop.conf.Configurable;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.BlockLocation;
Expand Down Expand Up @@ -107,6 +109,7 @@
import java.util.concurrent.atomic.AtomicBoolean;

import static com.amazonaws.regions.Regions.US_EAST_1;
import static com.amazonaws.services.s3.Headers.CRYPTO_KEYWRAP_ALGORITHM;
import static com.amazonaws.services.s3.Headers.SERVER_SIDE_ENCRYPTION;
import static com.amazonaws.services.s3.Headers.UNENCRYPTED_CONTENT_LENGTH;
import static com.amazonaws.services.s3.model.StorageClass.DeepArchive;
Expand Down Expand Up @@ -392,15 +395,27 @@ public FileStatus getFileStatus(Path path)
qualifiedPath(path));
}

private static long getObjectSize(Path path, ObjectMetadata metadata)
private long getObjectSize(Path path, ObjectMetadata metadata)
throws IOException
{
Map<String, String> userMetadata = metadata.getUserMetadata();
String length = userMetadata.get(UNENCRYPTED_CONTENT_LENGTH);
if (userMetadata.containsKey(SERVER_SIDE_ENCRYPTION) && length == null) {
throw new IOException(format("%s header is not set on an encrypted object: %s", UNENCRYPTED_CONTENT_LENGTH, path));
}
return (length != null) ? Long.parseLong(length) : metadata.getContentLength();

if (length != null) {
return Long.parseLong(length);
}

long reportedObjectSize = metadata.getContentLength();
// x-amz-unencrypted-content-length was not set, infer length for cse-kms encrypted objects by reading the tail until EOF
if (s3 instanceof AmazonS3Encryption && "kms".equalsIgnoreCase(userMetadata.get(CRYPTO_KEYWRAP_ALGORITHM))) {
try (FSDataInputStream in = open(path, FSDataInputStreamTail.MAX_SUPPORTED_PADDING_BYTES + 1)) {
return FSDataInputStreamTail.readTailForFileSize(path.toString(), reportedObjectSize, in);
}
}
return reportedObjectSize;
}

@Override
Expand Down

0 comments on commit 75f426e

Please sign in to comment.