|
@@ -52,6 +52,7 @@ import com.amazonaws.AmazonClientException;
|
|
import com.amazonaws.AmazonServiceException;
|
|
import com.amazonaws.AmazonServiceException;
|
|
import com.amazonaws.SdkBaseException;
|
|
import com.amazonaws.SdkBaseException;
|
|
import com.amazonaws.services.s3.AmazonS3;
|
|
import com.amazonaws.services.s3.AmazonS3;
|
|
|
|
+import com.amazonaws.services.s3.Headers;
|
|
import com.amazonaws.services.s3.model.CannedAccessControlList;
|
|
import com.amazonaws.services.s3.model.CannedAccessControlList;
|
|
import com.amazonaws.services.s3.model.CopyObjectRequest;
|
|
import com.amazonaws.services.s3.model.CopyObjectRequest;
|
|
import com.amazonaws.services.s3.model.DeleteObjectsRequest;
|
|
import com.amazonaws.services.s3.model.DeleteObjectsRequest;
|
|
@@ -80,6 +81,7 @@ import com.amazonaws.services.s3.transfer.model.UploadResult;
|
|
import com.amazonaws.event.ProgressListener;
|
|
import com.amazonaws.event.ProgressListener;
|
|
|
|
|
|
import org.apache.hadoop.fs.s3a.audit.AuditSpanS3A;
|
|
import org.apache.hadoop.fs.s3a.audit.AuditSpanS3A;
|
|
|
|
+import org.apache.hadoop.fs.statistics.impl.IOStatisticsStore;
|
|
import org.apache.hadoop.fs.store.audit.ActiveThreadSpanSource;
|
|
import org.apache.hadoop.fs.store.audit.ActiveThreadSpanSource;
|
|
import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting;
|
|
import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting;
|
|
import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
|
|
import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
|
|
@@ -212,6 +214,7 @@ import static org.apache.hadoop.fs.s3a.impl.CallableSupplier.submit;
|
|
import static org.apache.hadoop.fs.s3a.impl.CallableSupplier.waitForCompletionIgnoringExceptions;
|
|
import static org.apache.hadoop.fs.s3a.impl.CallableSupplier.waitForCompletionIgnoringExceptions;
|
|
import static org.apache.hadoop.fs.s3a.impl.ErrorTranslation.isObjectNotFound;
|
|
import static org.apache.hadoop.fs.s3a.impl.ErrorTranslation.isObjectNotFound;
|
|
import static org.apache.hadoop.fs.s3a.impl.ErrorTranslation.isUnknownBucket;
|
|
import static org.apache.hadoop.fs.s3a.impl.ErrorTranslation.isUnknownBucket;
|
|
|
|
+import static org.apache.hadoop.fs.s3a.impl.InternalConstants.CSE_PADDING_LENGTH;
|
|
import static org.apache.hadoop.fs.s3a.impl.InternalConstants.DEFAULT_UPLOAD_PART_COUNT_LIMIT;
|
|
import static org.apache.hadoop.fs.s3a.impl.InternalConstants.DEFAULT_UPLOAD_PART_COUNT_LIMIT;
|
|
import static org.apache.hadoop.fs.s3a.impl.InternalConstants.DELETE_CONSIDERED_IDEMPOTENT;
|
|
import static org.apache.hadoop.fs.s3a.impl.InternalConstants.DELETE_CONSIDERED_IDEMPOTENT;
|
|
import static org.apache.hadoop.fs.s3a.impl.InternalConstants.SC_404;
|
|
import static org.apache.hadoop.fs.s3a.impl.InternalConstants.SC_404;
|
|
@@ -356,6 +359,11 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
|
|
private AuditManagerS3A auditManager =
|
|
private AuditManagerS3A auditManager =
|
|
AuditIntegration.stubAuditManager();
|
|
AuditIntegration.stubAuditManager();
|
|
|
|
|
|
|
|
+ /**
|
|
|
|
+ * Is this S3A FS instance using S3 client side encryption?
|
|
|
|
+ */
|
|
|
|
+ private boolean isCSEEnabled;
|
|
|
|
+
|
|
/** Add any deprecated keys. */
|
|
/** Add any deprecated keys. */
|
|
@SuppressWarnings("deprecation")
|
|
@SuppressWarnings("deprecation")
|
|
private static void addDeprecatedKeys() {
|
|
private static void addDeprecatedKeys() {
|
|
@@ -413,12 +421,16 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
|
|
// DT Bindings may override this
|
|
// DT Bindings may override this
|
|
setEncryptionSecrets(new EncryptionSecrets(
|
|
setEncryptionSecrets(new EncryptionSecrets(
|
|
getEncryptionAlgorithm(bucket, conf),
|
|
getEncryptionAlgorithm(bucket, conf),
|
|
- getServerSideEncryptionKey(bucket, getConf())));
|
|
|
|
|
|
+ getS3EncryptionKey(bucket, getConf())));
|
|
|
|
|
|
invoker = new Invoker(new S3ARetryPolicy(getConf()), onRetry);
|
|
invoker = new Invoker(new S3ARetryPolicy(getConf()), onRetry);
|
|
instrumentation = new S3AInstrumentation(uri);
|
|
instrumentation = new S3AInstrumentation(uri);
|
|
initializeStatisticsBinding();
|
|
initializeStatisticsBinding();
|
|
-
|
|
|
|
|
|
+ // If CSE-KMS method is set then CSE is enabled.
|
|
|
|
+ isCSEEnabled = S3AUtils.lookupPassword(conf,
|
|
|
|
+ SERVER_SIDE_ENCRYPTION_ALGORITHM, null) != null;
|
|
|
|
+ LOG.debug("Client Side Encryption enabled: {}", isCSEEnabled);
|
|
|
|
+ setCSEGauge();
|
|
// Username is the current user at the time the FS was instantiated.
|
|
// Username is the current user at the time the FS was instantiated.
|
|
owner = UserGroupInformation.getCurrentUser();
|
|
owner = UserGroupInformation.getCurrentUser();
|
|
username = owner.getShortUserName();
|
|
username = owner.getShortUserName();
|
|
@@ -508,6 +520,10 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
|
|
blockFactory = S3ADataBlocks.createFactory(this, blockOutputBuffer);
|
|
blockFactory = S3ADataBlocks.createFactory(this, blockOutputBuffer);
|
|
blockOutputActiveBlocks = intOption(conf,
|
|
blockOutputActiveBlocks = intOption(conf,
|
|
FAST_UPLOAD_ACTIVE_BLOCKS, DEFAULT_FAST_UPLOAD_ACTIVE_BLOCKS, 1);
|
|
FAST_UPLOAD_ACTIVE_BLOCKS, DEFAULT_FAST_UPLOAD_ACTIVE_BLOCKS, 1);
|
|
|
|
+ // If CSE is enabled, do multipart uploads serially.
|
|
|
|
+ if (isCSEEnabled) {
|
|
|
|
+ blockOutputActiveBlocks = 1;
|
|
|
|
+ }
|
|
LOG.debug("Using S3ABlockOutputStream with buffer = {}; block={};" +
|
|
LOG.debug("Using S3ABlockOutputStream with buffer = {}; block={};" +
|
|
" queue limit={}",
|
|
" queue limit={}",
|
|
blockOutputBuffer, partSize, blockOutputActiveBlocks);
|
|
blockOutputBuffer, partSize, blockOutputActiveBlocks);
|
|
@@ -552,7 +568,22 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
|
|
stopAllServices();
|
|
stopAllServices();
|
|
throw e;
|
|
throw e;
|
|
}
|
|
}
|
|
|
|
+ }
|
|
|
|
|
|
|
|
+ /**
|
|
|
|
+ * Set the client side encryption gauge to 0 or 1, indicating if CSE is
|
|
|
|
+ * enabled through the gauge or not.
|
|
|
|
+ */
|
|
|
|
+ private void setCSEGauge() {
|
|
|
|
+ IOStatisticsStore ioStatisticsStore =
|
|
|
|
+ (IOStatisticsStore) getIOStatistics();
|
|
|
|
+ if (isCSEEnabled) {
|
|
|
|
+ ioStatisticsStore
|
|
|
|
+ .setGauge(CLIENT_SIDE_ENCRYPTION_ENABLED.getSymbol(), 1L);
|
|
|
|
+ } else {
|
|
|
|
+ ioStatisticsStore
|
|
|
|
+ .setGauge(CLIENT_SIDE_ENCRYPTION_ENABLED.getSymbol(), 0L);
|
|
|
|
+ }
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
@@ -1151,7 +1182,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
|
|
* Get the encryption algorithm of this endpoint.
|
|
* Get the encryption algorithm of this endpoint.
|
|
* @return the encryption algorithm.
|
|
* @return the encryption algorithm.
|
|
*/
|
|
*/
|
|
- public S3AEncryptionMethods getServerSideEncryptionAlgorithm() {
|
|
|
|
|
|
+ public S3AEncryptionMethods getS3EncryptionAlgorithm() {
|
|
return encryptionSecrets.getEncryptionMethod();
|
|
return encryptionSecrets.getEncryptionMethod();
|
|
}
|
|
}
|
|
|
|
|
|
@@ -1492,7 +1523,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
|
|
return new S3ObjectAttributes(bucket,
|
|
return new S3ObjectAttributes(bucket,
|
|
f,
|
|
f,
|
|
pathToKey(f),
|
|
pathToKey(f),
|
|
- getServerSideEncryptionAlgorithm(),
|
|
|
|
|
|
+ getS3EncryptionAlgorithm(),
|
|
encryptionSecrets.getEncryptionKey(),
|
|
encryptionSecrets.getEncryptionKey(),
|
|
eTag,
|
|
eTag,
|
|
versionId,
|
|
versionId,
|
|
@@ -1618,7 +1649,8 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
|
|
.withDowngradeSyncableExceptions(
|
|
.withDowngradeSyncableExceptions(
|
|
getConf().getBoolean(
|
|
getConf().getBoolean(
|
|
DOWNGRADE_SYNCABLE_EXCEPTIONS,
|
|
DOWNGRADE_SYNCABLE_EXCEPTIONS,
|
|
- DOWNGRADE_SYNCABLE_EXCEPTIONS_DEFAULT));
|
|
|
|
|
|
+ DOWNGRADE_SYNCABLE_EXCEPTIONS_DEFAULT))
|
|
|
|
+ .withCSEEnabled(isCSEEnabled);
|
|
return new FSDataOutputStream(
|
|
return new FSDataOutputStream(
|
|
new S3ABlockOutputStream(builder),
|
|
new S3ABlockOutputStream(builder),
|
|
null);
|
|
null);
|
|
@@ -3670,7 +3702,14 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
|
|
// look for the simple file
|
|
// look for the simple file
|
|
ObjectMetadata meta = getObjectMetadata(key);
|
|
ObjectMetadata meta = getObjectMetadata(key);
|
|
LOG.debug("Found exact file: normal file {}", key);
|
|
LOG.debug("Found exact file: normal file {}", key);
|
|
- return new S3AFileStatus(meta.getContentLength(),
|
|
|
|
|
|
+ long contentLength = meta.getContentLength();
|
|
|
|
+ // check if CSE is enabled, then strip padded length.
|
|
|
|
+ if (isCSEEnabled
|
|
|
|
+ && meta.getUserMetaDataOf(Headers.CRYPTO_CEK_ALGORITHM) != null
|
|
|
|
+ && contentLength >= CSE_PADDING_LENGTH) {
|
|
|
|
+ contentLength -= CSE_PADDING_LENGTH;
|
|
|
|
+ }
|
|
|
|
+ return new S3AFileStatus(contentLength,
|
|
dateToLong(meta.getLastModified()),
|
|
dateToLong(meta.getLastModified()),
|
|
path,
|
|
path,
|
|
getDefaultBlockSize(path),
|
|
getDefaultBlockSize(path),
|
|
@@ -4272,7 +4311,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
|
|
key, length, eTag, versionId);
|
|
key, length, eTag, versionId);
|
|
Path p = keyToQualifiedPath(key);
|
|
Path p = keyToQualifiedPath(key);
|
|
Preconditions.checkArgument(length >= 0, "content length is negative");
|
|
Preconditions.checkArgument(length >= 0, "content length is negative");
|
|
- final boolean isDir = objectRepresentsDirectory(key, length);
|
|
|
|
|
|
+ final boolean isDir = objectRepresentsDirectory(key);
|
|
// kick off an async delete
|
|
// kick off an async delete
|
|
CompletableFuture<?> deletion;
|
|
CompletableFuture<?> deletion;
|
|
if (!keepDirectoryMarkers(p)) {
|
|
if (!keepDirectoryMarkers(p)) {
|
|
@@ -4452,9 +4491,9 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
|
|
sb.append(", blockSize=").append(getDefaultBlockSize());
|
|
sb.append(", blockSize=").append(getDefaultBlockSize());
|
|
}
|
|
}
|
|
sb.append(", multiPartThreshold=").append(multiPartThreshold);
|
|
sb.append(", multiPartThreshold=").append(multiPartThreshold);
|
|
- if (getServerSideEncryptionAlgorithm() != null) {
|
|
|
|
- sb.append(", serverSideEncryptionAlgorithm='")
|
|
|
|
- .append(getServerSideEncryptionAlgorithm())
|
|
|
|
|
|
+ if (getS3EncryptionAlgorithm() != null) {
|
|
|
|
+ sb.append(", s3EncryptionAlgorithm='")
|
|
|
|
+ .append(getS3EncryptionAlgorithm())
|
|
.append('\'');
|
|
.append('\'');
|
|
}
|
|
}
|
|
if (blockFactory != null) {
|
|
if (blockFactory != null) {
|
|
@@ -4480,6 +4519,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
|
|
.append(getInstrumentation().toString())
|
|
.append(getInstrumentation().toString())
|
|
.append("}");
|
|
.append("}");
|
|
}
|
|
}
|
|
|
|
+ sb.append(", ClientSideEncryption=").append(isCSEEnabled);
|
|
sb.append('}');
|
|
sb.append('}');
|
|
return sb.toString();
|
|
return sb.toString();
|
|
}
|
|
}
|
|
@@ -5090,8 +5130,9 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
|
|
return isMagicCommitEnabled();
|
|
return isMagicCommitEnabled();
|
|
|
|
|
|
case SelectConstants.S3_SELECT_CAPABILITY:
|
|
case SelectConstants.S3_SELECT_CAPABILITY:
|
|
- // select is only supported if enabled
|
|
|
|
- return SelectBinding.isSelectEnabled(getConf());
|
|
|
|
|
|
+ // select is only supported if enabled and client side encryption is
|
|
|
|
+ // disabled.
|
|
|
|
+ return !isCSEEnabled && SelectBinding.isSelectEnabled(getConf());
|
|
|
|
|
|
case CommonPathCapabilities.FS_CHECKSUMS:
|
|
case CommonPathCapabilities.FS_CHECKSUMS:
|
|
// capability depends on FS configuration
|
|
// capability depends on FS configuration
|
|
@@ -5099,8 +5140,10 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
|
|
ETAG_CHECKSUM_ENABLED_DEFAULT);
|
|
ETAG_CHECKSUM_ENABLED_DEFAULT);
|
|
|
|
|
|
case CommonPathCapabilities.ABORTABLE_STREAM:
|
|
case CommonPathCapabilities.ABORTABLE_STREAM:
|
|
- case CommonPathCapabilities.FS_MULTIPART_UPLOADER:
|
|
|
|
return true;
|
|
return true;
|
|
|
|
+ case CommonPathCapabilities.FS_MULTIPART_UPLOADER:
|
|
|
|
+ // client side encryption doesn't support multipart uploader.
|
|
|
|
+ return !isCSEEnabled;
|
|
|
|
|
|
// this client is safe to use with buckets
|
|
// this client is safe to use with buckets
|
|
// containing directory markers anywhere in
|
|
// containing directory markers anywhere in
|
|
@@ -5236,7 +5279,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
|
|
*/
|
|
*/
|
|
private void requireSelectSupport(final Path source) throws
|
|
private void requireSelectSupport(final Path source) throws
|
|
UnsupportedOperationException {
|
|
UnsupportedOperationException {
|
|
- if (!SelectBinding.isSelectEnabled(getConf())) {
|
|
|
|
|
|
+ if (!isCSEEnabled && !SelectBinding.isSelectEnabled(getConf())) {
|
|
|
|
|
|
throw new UnsupportedOperationException(
|
|
throw new UnsupportedOperationException(
|
|
SelectConstants.SELECT_UNSUPPORTED);
|
|
SelectConstants.SELECT_UNSUPPORTED);
|
|
@@ -5359,6 +5402,10 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
|
|
public S3AMultipartUploaderBuilder createMultipartUploader(
|
|
public S3AMultipartUploaderBuilder createMultipartUploader(
|
|
final Path basePath)
|
|
final Path basePath)
|
|
throws IOException {
|
|
throws IOException {
|
|
|
|
+ if(isCSEEnabled) {
|
|
|
|
+ throw new UnsupportedOperationException("Multi-part uploader not "
|
|
|
|
+ + "supported for Client side encryption.");
|
|
|
|
+ }
|
|
final Path path = makeQualified(basePath);
|
|
final Path path = makeQualified(basePath);
|
|
try (AuditSpan span = entryPoint(MULTIPART_UPLOAD_INSTANTIATED, path)) {
|
|
try (AuditSpan span = entryPoint(MULTIPART_UPLOAD_INSTANTIATED, path)) {
|
|
StoreContext ctx = createStoreContext();
|
|
StoreContext ctx = createStoreContext();
|
|
@@ -5397,6 +5444,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
|
|
.setContextAccessors(new ContextAccessorsImpl())
|
|
.setContextAccessors(new ContextAccessorsImpl())
|
|
.setTimeProvider(getTtlTimeProvider())
|
|
.setTimeProvider(getTtlTimeProvider())
|
|
.setAuditor(getAuditor())
|
|
.setAuditor(getAuditor())
|
|
|
|
+ .setEnableCSE(isCSEEnabled)
|
|
.build();
|
|
.build();
|
|
}
|
|
}
|
|
|
|
|
|
@@ -5466,4 +5514,12 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
|
|
return S3AFileSystem.this.getRequestFactory();
|
|
return S3AFileSystem.this.getRequestFactory();
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
+
|
|
|
|
+ /**
|
|
|
|
+ * a method to know if Client side encryption is enabled or not.
|
|
|
|
+ * @return a boolean stating if CSE is enabled.
|
|
|
|
+ */
|
|
|
|
+ public boolean isCSEEnabled() {
|
|
|
|
+ return isCSEEnabled;
|
|
|
|
+ }
|
|
}
|
|
}
|