Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Add support for S3 EncryptionMaterialsProvider to PrestoS3FileSystem #3802

Closed
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
18 changes: 16 additions & 2 deletions pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -59,6 +59,8 @@
<air.test.jvmsize>2g</air.test.jvmsize>

<air.javadoc.lint>-missing</air.javadoc.lint>

<aws.sdk.version>1.9.31</aws.sdk.version>
</properties>

<modules>
Expand Down Expand Up @@ -577,8 +579,20 @@

<dependency>
<groupId>com.amazonaws</groupId>
<artifactId>aws-java-sdk</artifactId>
<version>1.8.9.1</version>
<artifactId>aws-java-sdk-core</artifactId>
<version>${aws.sdk.version}</version>
<exclusions>
<exclusion>
<groupId>commons-logging</groupId>
<artifactId>commons-logging</artifactId>
</exclusion>
</exclusions>
</dependency>

<dependency>
<groupId>com.amazonaws</groupId>
<artifactId>aws-java-sdk-s3</artifactId>
<version>${aws.sdk.version}</version>
<exclusions>
<exclusion>
<groupId>commons-logging</groupId>
Expand Down
13 changes: 12 additions & 1 deletion presto-hive/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -130,7 +130,18 @@

<dependency>
<groupId>com.amazonaws</groupId>
<artifactId>aws-java-sdk</artifactId>
<artifactId>aws-java-sdk-core</artifactId>
<exclusions>
<exclusion>
<groupId>joda-time</groupId>
<artifactId>joda-time</artifactId>
</exclusion>
</exclusions>
</dependency>

<dependency>
<groupId>com.amazonaws</groupId>
<artifactId>aws-java-sdk-s3</artifactId>
<exclusions>
<exclusion>
<groupId>joda-time</groupId>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -29,7 +29,10 @@
import com.amazonaws.regions.Regions;
import com.amazonaws.services.s3.AmazonS3;
import com.amazonaws.services.s3.AmazonS3Client;
import com.amazonaws.services.s3.AmazonS3EncryptionClient;
import com.amazonaws.services.s3.model.AmazonS3Exception;
import com.amazonaws.services.s3.model.CryptoConfiguration;
import com.amazonaws.services.s3.model.EncryptionMaterialsProvider;
import com.amazonaws.services.s3.model.GetObjectRequest;
import com.amazonaws.services.s3.model.ListObjectsRequest;
import com.amazonaws.services.s3.model.ObjectListing;
Expand All @@ -49,6 +52,7 @@
import io.airlift.log.Logger;
import io.airlift.units.DataSize;
import io.airlift.units.Duration;
import org.apache.hadoop.conf.Configurable;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.BlockLocation;
import org.apache.hadoop.fs.BufferedFSInputStream;
Expand Down Expand Up @@ -121,6 +125,7 @@ public static PrestoS3FileSystemStats getFileSystemStats()
public static final String S3_MULTIPART_MIN_FILE_SIZE = "presto.s3.multipart.min-file-size";
public static final String S3_MULTIPART_MIN_PART_SIZE = "presto.s3.multipart.min-part-size";
public static final String S3_USE_INSTANCE_CREDENTIALS = "presto.s3.use-instance-credentials";
public static final String S3_ENCRYPTION_MATERIALS_PROVIDER = "presto.s3.encryption-materials-provider";

private static final DataSize BLOCK_SIZE = new DataSize(32, MEGABYTE);
private static final DataSize MAX_SKIP_SIZE = new DataSize(1, MEGABYTE);
Expand Down Expand Up @@ -264,14 +269,20 @@ public FileStatus getFileStatus(Path path)
}

return new FileStatus(
metadata.getContentLength(),
getObjectSize(metadata),
false,
1,
BLOCK_SIZE.toBytes(),
lastModifiedTime(metadata),
qualifiedPath(path));
}

private static long getObjectSize(ObjectMetadata metadata)
{
String unencryptedContentLength = metadata.getUserMetadata().get("x-amz-unencrypted-content-length");
return unencryptedContentLength != null ? Long.parseLong(unencryptedContentLength) : metadata.getContentLength();
}

@Override
public FSDataInputStream open(Path path, int bufferSize)
throws IOException
Expand Down Expand Up @@ -444,6 +455,9 @@ private Iterator<LocatedFileStatus> statusFromPrefixes(List<String> prefixes)

private Iterator<LocatedFileStatus> statusFromObjects(List<S3ObjectSummary> objects)
{
// NOTE: for encrypted objects, S3ObjectSummary.size() used below is NOT correct,
// however, to get the correct size we'd need to make an additional request to get
// user metadata, and in this case it doesn't matter.
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Is this "doesn't matter" because the encrypted size is always larger? The Hive connector computes splits based on this size, so if the encrypted size was smaller, it would be possible to not generate a split for part of the file and thus miss some data.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Correct — doesn't matter because the encrypted size is always larger, and in this case since it's just used for the split calc. It matters most when trying to seek within a given file since seeking in S3 must be done relative to the unencrypted size (for instance, to read the last X bytes of a file to pull out metadata the reported size must be the unencrypted size).

return objects.stream()
.filter(object -> !object.getKey().endsWith("/"))
.map(object -> new FileStatus(
Expand Down Expand Up @@ -552,7 +566,14 @@ private static String keyFromPath(Path path)
private AmazonS3Client createAmazonS3Client(URI uri, Configuration hadoopConfig, ClientConfiguration clientConfig)
{
AWSCredentialsProvider credentials = getAwsCredentialsProvider(uri, hadoopConfig);
AmazonS3Client client = new AmazonS3Client(credentials, clientConfig, METRIC_COLLECTOR);
EncryptionMaterialsProvider emp = createEncryptionMaterialsProvider(hadoopConfig);
AmazonS3Client client;
if (emp != null) {
client = new AmazonS3EncryptionClient(credentials, emp, clientConfig, new CryptoConfiguration(), METRIC_COLLECTOR);
}
else {
client = new AmazonS3Client(credentials, clientConfig, METRIC_COLLECTOR);
}

// use local region when running inside of EC2
Region region = Regions.getCurrentRegion();
Expand All @@ -562,6 +583,26 @@ private AmazonS3Client createAmazonS3Client(URI uri, Configuration hadoopConfig,
return client;
}

private EncryptionMaterialsProvider createEncryptionMaterialsProvider(Configuration hadoopConfig)
{
EncryptionMaterialsProvider emp = null;
String empClassName = hadoopConfig.get(S3_ENCRYPTION_MATERIALS_PROVIDER);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Do you have a custom implementation of EncryptionMaterialsProvider?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yes — we implement an encryption materials provider that connects to our internal KMS, it reads initialization config from the hadoop config passed in. This works if you place your EMP jars in the $PRESTO_HOME/plugin/hive-hadoop2 directory.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Configuring this via the Hadoop config system doesn't fit with the way Presto and the Hive connector do configuration. We have first class config support in HiveClientConfig for most everything that a user would want to configure. All of these S3_ Hadoop config variables are for passing the Hive connector's configuration into PrestoS3FileSystem (because it is created by Hadoop and thus we don't have direct access to configure it). See HdfsConfigurationUpdater for where these are propagated from HiveClientConfig into the Hadoop Configuration object.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

OK – I didn't know about HiveClientConfig and HiveClientConfigurationUpdater, I'll make changes there to add the encryption materials provider classname. I would still pass the hadoop config into the EMP implementation though (if it implements Configurable), as the specific configs for a given EMP will be different for each implementation. Does that work?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I have a better understanding now. Since this requires a custom JAR and additional Hadoop configuration, we can add this as-is without the config going into HiveClientConfig. Thanks for the explanation!

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Nice! Thanks.

-nate

From: David Phillips <[email protected]mailto:[email protected]>
Reply-To: facebook/presto <[email protected]mailto:[email protected]>
Date: Tuesday, October 27, 2015 at 12:34 PM
To: facebook/presto <[email protected]mailto:[email protected]>
Cc: Nate Sammons <[email protected]mailto:[email protected]>
Subject: Re: [presto] Add support for S3 EncryptionMaterialsProvider to PrestoS3FileSystem (#3802)

In presto-hive/src/main/java/com/facebook/presto/hive/PrestoS3FileSystem.javahttps://github.com//pull/3802#discussion_r43166192:

@@ -562,6 +583,26 @@ private AmazonS3Client createAmazonS3Client(URI uri, Configuration hadoopConfig,
return client;
}

  • private EncryptionMaterialsProvider createEncryptionMaterialsProvider(Configuration hadoopConfig)
  • {
  •    EncryptionMaterialsProvider emp = null;
    
  •    String empClassName = hadoopConfig.get(S3_ENCRYPTION_MATERIALS_PROVIDER);
    

I have a better understanding now. Since this requires a custom JAR and additional Hadoop configuration, we can add this as-is without the config going into HiveClientConfig. Thanks for the explanation!


Reply to this email directly or view it on GitHubhttps://github.com//pull/3802/files#r43166192.


CONFIDENTIALITY NOTICE: This e-mail and any attachments are for the exclusive and confidential use of the intended recipient and may constitute non-public information. If you received this e-mail in error, disclosing, copying, distributing or taking any action in reliance of this e-mail is strictly prohibited and may be unlawful. Instead, please notify us immediately by return e-mail and promptly delete this message and its attachments from your computer system. We do not waive any work product or other applicable legal privilege(s) by the transmission of this message.


if (empClassName != null) {
try {
Class empClass = Class.forName(empClassName);
emp = (EncryptionMaterialsProvider) empClass.newInstance();
if (emp instanceof Configurable) {
((Configurable) emp).setConf(hadoopConfig);
}
}
catch (Exception x) {
throw new RuntimeException("Unable to load or create S3 encryption materials provider " + empClassName + ": " + x, x);
}
}

return emp;
}

private AWSCredentialsProvider getAwsCredentialsProvider(URI uri, Configuration conf)
{
// first try credentials from URI or static properties
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,7 @@
import com.amazonaws.services.s3.model.BucketLoggingConfiguration;
import com.amazonaws.services.s3.model.BucketNotificationConfiguration;
import com.amazonaws.services.s3.model.BucketPolicy;
import com.amazonaws.services.s3.model.BucketReplicationConfiguration;
import com.amazonaws.services.s3.model.BucketTaggingConfiguration;
import com.amazonaws.services.s3.model.BucketVersioningConfiguration;
import com.amazonaws.services.s3.model.BucketWebsiteConfiguration;
Expand Down Expand Up @@ -80,9 +81,11 @@
import com.amazonaws.services.s3.model.SetBucketLoggingConfigurationRequest;
import com.amazonaws.services.s3.model.SetBucketNotificationConfigurationRequest;
import com.amazonaws.services.s3.model.SetBucketPolicyRequest;
import com.amazonaws.services.s3.model.SetBucketReplicationConfigurationRequest;
import com.amazonaws.services.s3.model.SetBucketTaggingConfigurationRequest;
import com.amazonaws.services.s3.model.SetBucketVersioningConfigurationRequest;
import com.amazonaws.services.s3.model.SetBucketWebsiteConfigurationRequest;
import com.amazonaws.services.s3.model.SetObjectAclRequest;
import com.amazonaws.services.s3.model.StorageClass;
import com.amazonaws.services.s3.model.UploadPartRequest;
import com.amazonaws.services.s3.model.UploadPartResult;
Expand Down Expand Up @@ -763,4 +766,35 @@ public boolean isRequesterPaysEnabled(String bucketName)
{
return false;
}

@Override
public void setObjectAcl(SetObjectAclRequest setObjectAclRequest)
throws AmazonClientException, AmazonServiceException
{
}

@Override
public void setBucketReplicationConfiguration(String s, BucketReplicationConfiguration bucketReplicationConfiguration)
throws AmazonServiceException, AmazonClientException
{
}

@Override
public void setBucketReplicationConfiguration(SetBucketReplicationConfigurationRequest setBucketReplicationConfigurationRequest)
throws AmazonServiceException, AmazonClientException
{
}

@Override
public BucketReplicationConfiguration getBucketReplicationConfiguration(String s)
throws AmazonServiceException, AmazonClientException
{
return null;
}

@Override
public void deleteBucketReplicationConfiguration(String s)
throws AmazonServiceException, AmazonClientException
{
}
}