in nifi-extension-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/storage/PutGCSObject.java [332:544]
public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
FlowFile flowFile = session.get();
if (flowFile == null) {
return;
}
try {
final long startNanos = System.nanoTime();
final String bucket = context.getProperty(BUCKET)
.evaluateAttributeExpressions(flowFile)
.getValue();
final String key = context.getProperty(KEY)
.evaluateAttributeExpressions(flowFile)
.getValue();
final boolean overwrite = context.getProperty(OVERWRITE).asBoolean();
final FlowFile ff = flowFile;
final String ffFilename = ff.getAttributes().get(CoreAttributes.FILENAME.key());
final Map<String, String> attributes = new HashMap<>();
final ResourceTransferSource resourceTransferSource = context.getProperty(RESOURCE_TRANSFER_SOURCE).asAllowableValue(ResourceTransferSource.class);
final Storage storage = getCloudService();
try (final InputStream inputStream = getFileResource(resourceTransferSource, context, flowFile.getAttributes())
.map(FileResource::getInputStream)
.orElseGet(() -> session.read(ff))) {
final BlobId id = BlobId.of(bucket, key);
final BlobInfo.Builder blobInfoBuilder = BlobInfo.newBuilder(id);
final List<Storage.BlobWriteOption> blobWriteOptions = new ArrayList<>();
if (!overwrite) {
blobWriteOptions.add(Storage.BlobWriteOption.doesNotExist());
}
final String contentDispositionType = context.getProperty(CONTENT_DISPOSITION_TYPE).getValue();
if (contentDispositionType != null) {
blobInfoBuilder.setContentDisposition(contentDispositionType + "; filename=" + ffFilename);
}
final String contentType = context.getProperty(CONTENT_TYPE).evaluateAttributeExpressions(ff).getValue();
if (contentType != null) {
blobInfoBuilder.setContentType(contentType);
}
final String crc32c = context.getProperty(CRC32C).evaluateAttributeExpressions(ff).getValue();
if (crc32c != null) {
blobInfoBuilder.setCrc32c(crc32c);
blobWriteOptions.add(Storage.BlobWriteOption.crc32cMatch());
}
final String acl = context.getProperty(ACL).getValue();
if (acl != null) {
blobWriteOptions.add(Storage.BlobWriteOption.predefinedAcl(
Storage.PredefinedAcl.valueOf(acl)
));
}
final String encryptionKey = context.getProperty(ENCRYPTION_KEY)
.evaluateAttributeExpressions(ff).getValue();
if (encryptionKey != null) {
blobWriteOptions.add(Storage.BlobWriteOption.encryptionKey(encryptionKey));
}
final boolean gzipCompress = context.getProperty(GZIPCONTENT).asBoolean();
if (!gzipCompress) {
blobWriteOptions.add(Storage.BlobWriteOption.disableGzipContent());
}
final HashMap<String, String> userMetadata = new HashMap<>();
for (final Map.Entry<PropertyDescriptor, String> entry : context.getProperties().entrySet()) {
if (entry.getKey().isDynamic()) {
final String value = context.getProperty(
entry.getKey()).evaluateAttributeExpressions(ff).getValue();
userMetadata.put(entry.getKey().getName(), value);
}
}
if (!userMetadata.isEmpty()) {
blobInfoBuilder.setMetadata(userMetadata);
}
try {
final Blob blob = storage.createFrom(blobInfoBuilder.build(),
inputStream,
blobWriteOptions.toArray(new Storage.BlobWriteOption[blobWriteOptions.size()])
);
// Create attributes
attributes.put(BUCKET_ATTR, blob.getBucket());
attributes.put(KEY_ATTR, blob.getName());
if (blob.getSize() != null) {
attributes.put(SIZE_ATTR, String.valueOf(blob.getSize()));
}
if (blob.getCacheControl() != null) {
attributes.put(CACHE_CONTROL_ATTR, blob.getCacheControl());
}
if (blob.getComponentCount() != null) {
attributes.put(COMPONENT_COUNT_ATTR, String.valueOf(blob.getComponentCount()));
}
if (blob.getContentDisposition() != null) {
attributes.put(CONTENT_DISPOSITION_ATTR, blob.getContentDisposition());
final Util.ParsedContentDisposition parsed = Util.parseContentDisposition(blob.getContentDisposition());
if (parsed != null) {
attributes.put(CoreAttributes.FILENAME.key(), parsed.getFileName());
}
}
if (blob.getContentEncoding() != null) {
attributes.put(CONTENT_ENCODING_ATTR, blob.getContentEncoding());
}
if (blob.getContentLanguage() != null) {
attributes.put(CONTENT_LANGUAGE_ATTR, blob.getContentLanguage());
}
if (blob.getContentType() != null) {
attributes.put(CoreAttributes.MIME_TYPE.key(), blob.getContentType());
}
if (blob.getCrc32c() != null) {
attributes.put(CRC32C_ATTR, blob.getCrc32c());
}
if (blob.getCustomerEncryption() != null) {
final BlobInfo.CustomerEncryption encryption = blob.getCustomerEncryption();
attributes.put(ENCRYPTION_ALGORITHM_ATTR, encryption.getEncryptionAlgorithm());
attributes.put(ENCRYPTION_SHA256_ATTR, encryption.getKeySha256());
}
if (blob.getEtag() != null) {
attributes.put(ETAG_ATTR, blob.getEtag());
}
if (blob.getGeneratedId() != null) {
attributes.put(GENERATED_ID_ATTR, blob.getGeneratedId());
}
if (blob.getGeneration() != null) {
attributes.put(GENERATION_ATTR, String.valueOf(blob.getGeneration()));
}
if (blob.getMd5() != null) {
attributes.put(MD5_ATTR, blob.getMd5());
}
if (blob.getMediaLink() != null) {
attributes.put(MEDIA_LINK_ATTR, blob.getMediaLink());
}
if (blob.getMetageneration() != null) {
attributes.put(METAGENERATION_ATTR, String.valueOf(blob.getMetageneration()));
}
if (blob.getOwner() != null) {
final Acl.Entity entity = blob.getOwner();
if (entity instanceof Acl.User) {
attributes.put(OWNER_ATTR, ((Acl.User) entity).getEmail());
attributes.put(OWNER_TYPE_ATTR, "user");
} else if (entity instanceof Acl.Group) {
attributes.put(OWNER_ATTR, ((Acl.Group) entity).getEmail());
attributes.put(OWNER_TYPE_ATTR, "group");
} else if (entity instanceof Acl.Domain) {
attributes.put(OWNER_ATTR, ((Acl.Domain) entity).getDomain());
attributes.put(OWNER_TYPE_ATTR, "domain");
} else if (entity instanceof Acl.Project) {
attributes.put(OWNER_ATTR, ((Acl.Project) entity).getProjectId());
attributes.put(OWNER_TYPE_ATTR, "project");
}
}
if (blob.getSelfLink() != null) {
attributes.put(URI_ATTR, blob.getSelfLink());
}
if (blob.getCreateTimeOffsetDateTime() != null) {
attributes.put(CREATE_TIME_ATTR, String.valueOf(blob.getCreateTimeOffsetDateTime().toInstant().toEpochMilli()));
}
if (blob.getUpdateTimeOffsetDateTime() != null) {
attributes.put(UPDATE_TIME_ATTR, String.valueOf(blob.getUpdateTimeOffsetDateTime().toInstant().toEpochMilli()));
}
} catch (StorageException | IOException e) {
getLogger().error("Failure completing upload flowfile={} bucket={} key={} reason={}",
ffFilename, bucket, key, e.getMessage(), e);
throw (e);
}
}
if (!attributes.isEmpty()) {
flowFile = session.putAllAttributes(flowFile, attributes);
}
session.transfer(flowFile, REL_SUCCESS);
final long millis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startNanos);
final String transitUri = getTransitUri(storage.getOptions().getHost(), bucket, key);
session.getProvenanceReporter().send(flowFile, transitUri, millis);
getLogger().info("Successfully put {} to Google Cloud Storage in {} milliseconds", ff, millis);
} catch (final ProcessException | StorageException | IOException e) {
getLogger().error("Failed to put {} to Google Cloud Storage due to {}", flowFile, e.getMessage(), e);
flowFile = session.penalize(flowFile);
session.transfer(flowFile, REL_FAILURE);
}
}