forked from apache/flink
-
Notifications
You must be signed in to change notification settings - Fork 0
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
[FLINK-24228][connectors/firehose] Amazon Kinesis Data Firehose sink …
…based on the Async Sink Base implemented: - Refactored AWSUnifiedSinksUtil into a class that caters for Kinesis and Firehose - Extracted commonalities between KDS & KDF sinks into flink-connector-aws-base - Implemented integration test based on Localstack container - Changing host/container ports to be different, changing HTTP1.1 to being the default, localstack issue fixed - Added docs page, changed type in Firehose, turned logging off, removed unused dependencies.
- Loading branch information
1 parent
8a9a08b
commit 0e908be
Showing
36 changed files
with
2,011 additions
and
265 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
129 changes: 129 additions & 0 deletions
129
...aws-base/src/test/java/org/apache/flink/connector/aws/testutils/AWSServicesTestUtils.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,129 @@ | ||
/* | ||
* 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.flink.connector.aws.testutils; | ||
|
||
import org.apache.flink.connector.aws.config.AWSConfigConstants; | ||
import org.apache.flink.connector.aws.util.AWSGeneralUtil; | ||
|
||
import software.amazon.awssdk.auth.credentials.AwsBasicCredentials; | ||
import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider; | ||
import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider; | ||
import software.amazon.awssdk.core.waiters.WaiterResponse; | ||
import software.amazon.awssdk.http.async.SdkAsyncHttpClient; | ||
import software.amazon.awssdk.regions.Region; | ||
import software.amazon.awssdk.services.iam.IamAsyncClient; | ||
import software.amazon.awssdk.services.iam.model.CreateRoleRequest; | ||
import software.amazon.awssdk.services.iam.model.CreateRoleResponse; | ||
import software.amazon.awssdk.services.s3.S3AsyncClient; | ||
import software.amazon.awssdk.services.s3.model.CreateBucketRequest; | ||
import software.amazon.awssdk.services.s3.model.HeadBucketRequest; | ||
import software.amazon.awssdk.services.s3.model.HeadBucketResponse; | ||
import software.amazon.awssdk.services.s3.model.ListObjectsRequest; | ||
import software.amazon.awssdk.services.s3.model.ListObjectsResponse; | ||
import software.amazon.awssdk.services.s3.model.S3Object; | ||
import software.amazon.awssdk.services.s3.waiters.S3AsyncWaiter; | ||
|
||
import java.net.URI; | ||
import java.net.URISyntaxException; | ||
import java.util.List; | ||
import java.util.Properties; | ||
import java.util.concurrent.CompletableFuture; | ||
import java.util.concurrent.ExecutionException; | ||
|
||
import static org.apache.flink.connector.aws.config.AWSConfigConstants.AWS_CREDENTIALS_PROVIDER; | ||
import static org.apache.flink.connector.aws.config.AWSConfigConstants.AWS_ENDPOINT; | ||
import static org.apache.flink.connector.aws.config.AWSConfigConstants.AWS_REGION; | ||
import static org.apache.flink.connector.aws.config.AWSConfigConstants.TRUST_ALL_CERTIFICATES; | ||
|
||
/** | ||
* A set of static methods that can be used to call common AWS services on the Localstack container. | ||
*/ | ||
public class AWSServicesTestUtils { | ||
|
||
private static final String ACCESS_KEY_ID = "accessKeyId"; | ||
private static final String SECRET_ACCESS_KEY = "secretAccessKey"; | ||
|
||
public static S3AsyncClient getS3Client(String endpoint) throws URISyntaxException { | ||
return S3AsyncClient.builder() | ||
.httpClient(getHttpClient(endpoint)) | ||
.region(Region.AP_SOUTHEAST_1) | ||
.endpointOverride(new URI(endpoint)) | ||
.credentialsProvider(getDefaultCredentials()) | ||
.build(); | ||
} | ||
|
||
public static IamAsyncClient getIamClient(String endpoint) throws URISyntaxException { | ||
return IamAsyncClient.builder() | ||
.httpClient(getHttpClient(endpoint)) | ||
.region(Region.AWS_GLOBAL) | ||
.endpointOverride(new URI(endpoint)) | ||
.credentialsProvider(getDefaultCredentials()) | ||
.build(); | ||
} | ||
|
||
public static AwsCredentialsProvider getDefaultCredentials() { | ||
return StaticCredentialsProvider.create( | ||
AwsBasicCredentials.create(ACCESS_KEY_ID, SECRET_ACCESS_KEY)); | ||
} | ||
|
||
public static Properties getConfig(String endpoint) { | ||
Properties config = new Properties(); | ||
config.setProperty(AWS_REGION, Region.AP_SOUTHEAST_1.toString()); | ||
config.setProperty(AWS_ENDPOINT, endpoint); | ||
config.setProperty(AWSConfigConstants.accessKeyId(AWS_CREDENTIALS_PROVIDER), ACCESS_KEY_ID); | ||
config.setProperty( | ||
AWSConfigConstants.secretKey(AWS_CREDENTIALS_PROVIDER), SECRET_ACCESS_KEY); | ||
config.setProperty(TRUST_ALL_CERTIFICATES, "true"); | ||
return config; | ||
} | ||
|
||
public static SdkAsyncHttpClient getHttpClient(String endpoint) { | ||
return AWSGeneralUtil.createAsyncHttpClient(getConfig(endpoint)); | ||
} | ||
|
||
public static void createBucket(S3AsyncClient s3Client, String bucketName) | ||
throws ExecutionException, InterruptedException { | ||
CreateBucketRequest bucketRequest = | ||
CreateBucketRequest.builder().bucket(bucketName).build(); | ||
s3Client.createBucket(bucketRequest); | ||
|
||
HeadBucketRequest bucketRequestWait = | ||
HeadBucketRequest.builder().bucket(bucketName).build(); | ||
|
||
S3AsyncWaiter s3Waiter = s3Client.waiter(); | ||
CompletableFuture<WaiterResponse<HeadBucketResponse>> waiterResponseFuture = | ||
s3Waiter.waitUntilBucketExists(bucketRequestWait); | ||
|
||
waiterResponseFuture.get(); | ||
} | ||
|
||
public static void createIAMRole(IamAsyncClient iam, String roleName) | ||
throws ExecutionException, InterruptedException { | ||
CreateRoleRequest request = CreateRoleRequest.builder().roleName(roleName).build(); | ||
|
||
CompletableFuture<CreateRoleResponse> responseFuture = iam.createRole(request); | ||
responseFuture.get(); | ||
} | ||
|
||
public static List<S3Object> listBucketObjects(S3AsyncClient s3, String bucketName) | ||
throws ExecutionException, InterruptedException { | ||
ListObjectsRequest listObjects = ListObjectsRequest.builder().bucket(bucketName).build(); | ||
CompletableFuture<ListObjectsResponse> res = s3.listObjects(listObjects); | ||
return res.get().contents(); | ||
} | ||
} |
Oops, something went wrong.