You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@beam.apache.org by mm...@apache.org on 2022/07/08 07:00:30 UTC
[beam] branch master updated: Deprecate AWS IOs (Java) using AWS SDK v1 in favor of IOs in amazon-web-services2 (#22093)
This is an automated email from the ASF dual-hosted git repository.
mmack pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/beam.git
The following commit(s) were added to refs/heads/master by this push:
new f5435c05758 Deprecate AWS IOs (Java) using AWS SDK v1 in favor of IOs in amazon-web-services2 (#22093)
f5435c05758 is described below
commit f5435c0575870062f39575271c0f483117908403
Author: Moritz Mack <mm...@talend.com>
AuthorDate: Fri Jul 8 09:00:22 2022 +0200
Deprecate AWS IOs (Java) using AWS SDK v1 in favor of IOs in amazon-web-services2 (#22093)
---
CHANGES.md | 5 ++++-
.../beam/sdk/io/aws/dynamodb/DynamoDBIO.java | 15 +++++++++++++
.../apache/beam/sdk/io/aws/s3/S3FileSystem.java | 8 +++++++
.../java/org/apache/beam/sdk/io/aws/sns/SnsIO.java | 10 +++++++++
.../java/org/apache/beam/sdk/io/aws/sqs/SqsIO.java | 15 +++++++++++++
.../org/apache/beam/sdk/io/kinesis/KinesisIO.java | 13 +++++++++++
website/www/site/data/io_matrix.yaml | 26 ++++------------------
7 files changed, 69 insertions(+), 23 deletions(-)
diff --git a/CHANGES.md b/CHANGES.md
index c740a47e640..a2d47753afe 100644
--- a/CHANGES.md
+++ b/CHANGES.md
@@ -74,7 +74,10 @@
## Deprecations
* Support for Spark 2.4.x is deprecated and will be dropped with the release of Beam 2.44.0 or soon after (Spark runner) ([#22094](https://github.com/apache/beam/issues/22094)).
-* X behavior is deprecated and will be removed in X versions ([#X](https://github.com/apache/beam/issues/X)).
+* The modules [amazon-web-services](https://github.com/apache/beam/tree/master/sdks/java/io/amazon-web-services) and
+ [kinesis](https://github.com/apache/beam/tree/master/sdks/java/io/kinesis) for AWS Java SDK v1 are deprecated
+ in favor of [amazon-web-services2](https://github.com/apache/beam/tree/master/sdks/java/io/amazon-web-services2)
+ and will be eventually removed after a few Beam releases (Java) ([#21249](https://github.com/apache/beam/issues/21249)).
## Bugfixes
diff --git a/sdks/java/io/amazon-web-services/src/main/java/org/apache/beam/sdk/io/aws/dynamodb/DynamoDBIO.java b/sdks/java/io/amazon-web-services/src/main/java/org/apache/beam/sdk/io/aws/dynamodb/DynamoDBIO.java
index 2e2d09ea56b..9f0928ae8db 100644
--- a/sdks/java/io/amazon-web-services/src/main/java/org/apache/beam/sdk/io/aws/dynamodb/DynamoDBIO.java
+++ b/sdks/java/io/amazon-web-services/src/main/java/org/apache/beam/sdk/io/aws/dynamodb/DynamoDBIO.java
@@ -131,8 +131,13 @@ import org.slf4j.LoggerFactory;
* <li>ScanRequestFn, which you build a ScanRequest object with at least table name and total
* number of segment. Note This number should base on the number of your workers
* </ul>
+ *
+ * @deprecated Module <code>beam-sdks-java-io-amazon-web-services</code> is deprecated and will be
+ * eventually removed. Please migrate to {@link org.apache.beam.sdk.io.aws2.dynamodb.DynamoDBIO}
+ * in module <code>beam-sdks-java-io-amazon-web-services2</code>.
*/
@Experimental(Kind.SOURCE_SINK)
+@Deprecated
@SuppressWarnings({
"nullness" // TODO(https://github.com/apache/beam/issues/20497)
})
@@ -227,6 +232,11 @@ public final class DynamoDBIO {
@Override
public PCollection<T> expand(PBegin input) {
+ LoggerFactory.getLogger(DynamoDBIO.class)
+ .warn(
+ "You are using a deprecated IO for DynamoDB. Please migrate to module "
+ + "'org.apache.beam:beam-sdks-java-io-amazon-web-services2'.");
+
checkArgument((getScanRequestFn() != null), "withScanRequestFn() is required");
checkArgument((getAwsClientsProvider() != null), "withAwsClientsProvider() is required");
ScanRequest scanRequest = getScanRequestFn().apply(null);
@@ -454,6 +464,11 @@ public final class DynamoDBIO {
@Override
public PCollection<Void> expand(PCollection<T> input) {
+ LoggerFactory.getLogger(DynamoDBIO.class)
+ .warn(
+ "You are using a deprecated IO for DynamoDB. Please migrate to module "
+ + "'org.apache.beam:beam-sdks-java-io-amazon-web-services2'.");
+
return input.apply(ParDo.of(new WriteFn<>(this)));
}
diff --git a/sdks/java/io/amazon-web-services/src/main/java/org/apache/beam/sdk/io/aws/s3/S3FileSystem.java b/sdks/java/io/amazon-web-services/src/main/java/org/apache/beam/sdk/io/aws/s3/S3FileSystem.java
index 6e9d60e0250..63eb83f30e4 100644
--- a/sdks/java/io/amazon-web-services/src/main/java/org/apache/beam/sdk/io/aws/s3/S3FileSystem.java
+++ b/sdks/java/io/amazon-web-services/src/main/java/org/apache/beam/sdk/io/aws/s3/S3FileSystem.java
@@ -86,7 +86,11 @@ import org.slf4j.LoggerFactory;
* {@link FileSystem} implementation for storage systems that use the S3 protocol.
*
* @see S3FileSystemSchemeRegistrar
+ * @deprecated Module <code>beam-sdks-java-io-amazon-web-services</code> is deprecated and will be
+ * eventually removed. Please migrate to module <code>beam-sdks-java-io-amazon-web-services2
+ * </code>.
*/
+@Deprecated
@SuppressWarnings({
"nullness" // TODO(https://github.com/apache/beam/issues/20497)
})
@@ -120,6 +124,10 @@ class S3FileSystem extends FileSystem<S3ResourceId> {
MoreExecutors.listeningDecorator(
Executors.newFixedThreadPool(
config.getS3ThreadPoolSize(), new ThreadFactoryBuilder().setDaemon(true).build()));
+
+ LOG.warn(
+ "You are using a deprecated file system for S3. Please migrate to module "
+ + "'org.apache.beam:beam-sdks-java-io-amazon-web-services2'.");
}
S3FileSystem(S3Options options) {
diff --git a/sdks/java/io/amazon-web-services/src/main/java/org/apache/beam/sdk/io/aws/sns/SnsIO.java b/sdks/java/io/amazon-web-services/src/main/java/org/apache/beam/sdk/io/aws/sns/SnsIO.java
index 308305a4dd4..2e734d641ea 100644
--- a/sdks/java/io/amazon-web-services/src/main/java/org/apache/beam/sdk/io/aws/sns/SnsIO.java
+++ b/sdks/java/io/amazon-web-services/src/main/java/org/apache/beam/sdk/io/aws/sns/SnsIO.java
@@ -85,8 +85,13 @@ import org.slf4j.LoggerFactory;
* If you need the full ResponseMetadata and SdkHttpMetadata you can call {@link
* Write#withFullPublishResult}. If you need the HTTP status code but not the response headers you
* can call {@link Write#withFullPublishResultWithoutHeaders}.
+ *
+ * @deprecated Module <code>beam-sdks-java-io-amazon-web-services</code> is deprecated and will be
+ * eventually removed. Please migrate to {@link org.apache.beam.sdk.io.aws2.sns.SnsIO} in module
+ * <code>beam-sdks-java-io-amazon-web-services2</code>.
*/
@Experimental(Kind.SOURCE_SINK)
+@Deprecated
@SuppressWarnings({
"nullness" // TODO(https://github.com/apache/beam/issues/20497)
})
@@ -308,6 +313,11 @@ public final class SnsIO {
@Override
public PCollectionTuple expand(PCollection<PublishRequest> input) {
+ LoggerFactory.getLogger(SnsIO.class)
+ .warn(
+ "You are using a deprecated IO for Sns. Please migrate to module "
+ + "'org.apache.beam:beam-sdks-java-io-amazon-web-services2'.");
+
checkArgument(getTopicName() != null, "withTopicName() is required");
PCollectionTuple result =
input.apply(
diff --git a/sdks/java/io/amazon-web-services/src/main/java/org/apache/beam/sdk/io/aws/sqs/SqsIO.java b/sdks/java/io/amazon-web-services/src/main/java/org/apache/beam/sdk/io/aws/sqs/SqsIO.java
index 5f214f2e47d..e430261daeb 100644
--- a/sdks/java/io/amazon-web-services/src/main/java/org/apache/beam/sdk/io/aws/sqs/SqsIO.java
+++ b/sdks/java/io/amazon-web-services/src/main/java/org/apache/beam/sdk/io/aws/sqs/SqsIO.java
@@ -36,6 +36,7 @@ import org.apache.beam.sdk.values.PCollection;
import org.apache.beam.sdk.values.PDone;
import org.checkerframework.checker.nullness.qual.Nullable;
import org.joda.time.Duration;
+import org.slf4j.LoggerFactory;
/**
* An unbounded source for Amazon Simple Queue Service (SQS).
@@ -77,8 +78,13 @@ import org.joda.time.Duration;
* }</pre>
*
* <p>For more information on the available options see {@link AwsOptions}.
+ *
+ * @deprecated Module <code>beam-sdks-java-io-amazon-web-services</code> is deprecated and will be
+ * eventually removed. Please migrate to {@link org.apache.beam.sdk.io.aws2.sqs.SqsIO} in module
+ * <code>beam-sdks-java-io-amazon-web-services2</code>.
*/
@Experimental(Kind.SOURCE_SINK)
+@Deprecated
@SuppressWarnings({
"nullness" // TODO(https://github.com/apache/beam/issues/20497)
})
@@ -164,6 +170,10 @@ public class SqsIO {
@Override
public PCollection<Message> expand(PBegin input) {
+ LoggerFactory.getLogger(SqsIO.class)
+ .warn(
+ "You are using a deprecated IO for Sqs. Please migrate to module "
+ + "'org.apache.beam:beam-sdks-java-io-amazon-web-services2'.");
org.apache.beam.sdk.io.Read.Unbounded<Message> unbounded =
org.apache.beam.sdk.io.Read.from(
@@ -197,6 +207,11 @@ public class SqsIO {
@Override
public PDone expand(PCollection<SendMessageRequest> input) {
+ LoggerFactory.getLogger(SqsIO.class)
+ .warn(
+ "You are using a deprecated IO for Sqs. Please migrate to module "
+ + "'org.apache.beam:beam-sdks-java-io-amazon-web-services2'.");
+
input.apply(
ParDo.of(
new SqsWriteFn(
diff --git a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisIO.java b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisIO.java
index e07cd3b7dd2..637bd84b7a6 100644
--- a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisIO.java
+++ b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisIO.java
@@ -314,11 +314,16 @@ import org.slf4j.LoggerFactory;
* <p>For more information about configuratiom parameters, see the <a
* href="https://github.com/awslabs/amazon-kinesis-producer/blob/master/java/amazon-kinesis-producer-sample/default_config.properties">sample
* of configuration file</a>.
+ *
+ * @deprecated Module <code>beam-sdks-java-io-kinesis</code> is deprecated and will be eventually
+ * removed. Please migrate to {@link org.apache.beam.sdk.io.aws2.kinesis.KinesisIO} in module
+ * <code>beam-sdks-java-io-amazon-web-services2</code>.
*/
@Experimental(Kind.SOURCE_SINK)
@SuppressWarnings({
"nullness" // TODO(https://github.com/apache/beam/issues/20497)
})
+@Deprecated
public final class KinesisIO {
private static final Logger LOG = LoggerFactory.getLogger(KinesisIO.class);
@@ -655,6 +660,10 @@ public final class KinesisIO {
@Override
public PCollection<T> expand(PBegin input) {
+ LOG.warn(
+ "You are using a deprecated IO for Kinesis. Please migrate to module "
+ + "'org.apache.beam:beam-sdks-java-io-amazon-web-services2'.");
+
Unbounded<KinesisRecord> unbounded =
org.apache.beam.sdk.io.Read.from(
new KinesisSource(
@@ -875,6 +884,10 @@ public final class KinesisIO {
@Override
public PDone expand(PCollection<byte[]> input) {
+ LOG.warn(
+ "You are using a deprecated IO for Kinesis. Please migrate to module "
+ + "'org.apache.beam:beam-sdks-java-io-amazon-web-services2'.");
+
checkArgument(getStreamName() != null, "withStreamName() is required");
checkArgument(
(getPartitionKey() != null) || (getPartitioner() != null),
diff --git a/website/www/site/data/io_matrix.yaml b/website/www/site/data/io_matrix.yaml
index 1527bb0623d..a0e47c294bb 100644
--- a/website/www/site/data/io_matrix.yaml
+++ b/website/www/site/data/io_matrix.yaml
@@ -136,11 +136,8 @@ categories:
description: "`FileSystem` implementation for [Amazon S3](https://aws.amazon.com/s3/)."
implementations:
- language: java
- name: org.apache.beam.sdk.io.aws2.s3.S3FileSystemRegistrar (recommended)
+ name: org.apache.beam.sdk.io.aws2.s3.S3FileSystemRegistrar
url: https://beam.apache.org/releases/javadoc/current/org/apache/beam/sdk/io/aws2/s3/S3FileSystemRegistrar.html
- - language: java
- name: org.apache.beam.sdk.io.aws.s3.S3FileSystemRegistrar
- url: https://beam.apache.org/releases/javadoc/current/org/apache/beam/sdk/io/aws/s3/S3FileSystemRegistrar.html
- transform: In-memory
description: "`FileSystem` implementation in memory; useful for testing."
implementations:
@@ -154,11 +151,8 @@ categories:
description: PTransforms for reading from and writing to [Kinesis](https://aws.amazon.com/kinesis/) streams.
implementations:
- language: java
- name: org.apache.beam.sdk.io.aws2.kinesis.KinesisIO (recommended)
+ name: org.apache.beam.sdk.io.aws2.kinesis.KinesisIO
url: https://beam.apache.org/releases/javadoc/current/org/apache/beam/sdk/io/aws2/kinesis/KinesisIO.html
- - language: java
- name: org.apache.beam.sdk.io.kinesis.KinesisIO
- url: https://beam.apache.org/releases/javadoc/current/org/apache/beam/sdk/io/kinesis/KinesisIO.html
- transform: AmqpIO
description: AMQP 1.0 protocol using the Apache QPid Proton-J library
implementations:
@@ -211,20 +205,11 @@ categories:
description: An unbounded source for [Amazon Simple Queue Service (SQS)](https://aws.amazon.com/sqs/).
implementations:
- language: java
- name: org.apache.beam.sdk.io.aws2.sqs.SqsIO (recommended)
+ name: org.apache.beam.sdk.io.aws2.sqs.SqsIO
url: https://beam.apache.org/releases/javadoc/current/org/apache/beam/sdk/io/aws2/sqs/SqsIO.html
- - language: java
- name: org.apache.beam.sdk.io.aws.sqs.SqsIO
- url: https://beam.apache.org/releases/javadoc/current/org/apache/beam/sdk/io/aws/sqs/SqsIO.html
- transform: SnsIO
description: PTransforms for writing to [Amazon Simple Notification Service (SNS)](https://aws.amazon.com/sns/).
implementations:
- - language: java
- name: org.apache.beam.sdk.io.aws2.sns.SnsIO (recommended)
- url: https://beam.apache.org/releases/javadoc/current/org/apache/beam/sdk/io/aws2/sns/SnsIO.html
- - language: java
- name: org.apache.beam.sdk.io.aws.sns.SnsIO
- url: https://beam.apache.org/releases/javadoc/current/org/apache/beam/sdk/io/aws/sns/SnsIO.html
- language: java
name: org.apache.beam.sdk.io.aws2.sns.SnsIO
url: https://beam.apache.org/releases/javadoc/current/org/apache/beam/sdk/io/aws2/sns/SnsIO.html
@@ -359,11 +344,8 @@ categories:
description: Read from and write to [Amazon DynamoDB](https://aws.amazon.com/dynamodb/).
implementations:
- language: java
- name: org.apache.beam.sdk.io.aws2.dynamodb.DynamoDBIO (recommended)
+ name: org.apache.beam.sdk.io.aws2.dynamodb.DynamoDBIO
url: https://beam.apache.org/releases/javadoc/current/org/apache/beam/sdk/io/aws2/dynamodb/DynamoDBIO.html
- - language: java
- name: org.apache.beam.sdk.io.aws.dynamodb.DynamoDBIO
- url: https://beam.apache.org/releases/javadoc/current/org/apache/beam/sdk/io/aws/dynamodb/DynamoDBIO.html
- transform: ClickHouseIO
description: Transform for writing to [ClickHouse](https://clickhouse.tech/).
implementations: