TheNeuralBit commented on a change in pull request #12297: URL: https://github.com/apache/beam/pull/12297#discussion_r459138094
########## File path: sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisReadTransformRegistrar.java ########## @@ -0,0 +1,175 @@ +/* + * 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.beam.sdk.io.kinesis; + +import com.amazonaws.regions.Regions; +import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream; +import com.google.auto.service.AutoService; +import java.util.Map; +import org.apache.beam.sdk.annotations.Experimental; +import org.apache.beam.sdk.annotations.Experimental.Kind; +import org.apache.beam.sdk.expansion.ExternalTransformRegistrar; +import org.apache.beam.sdk.transforms.ExternalTransformBuilder; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.values.PBegin; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap; +import org.joda.time.Duration; +import org.joda.time.Instant; + +/** Exposes {@link KinesisIO.Read} as an external transform for cross-language usage. */ +@Experimental(Kind.PORTABILITY) +@AutoService(ExternalTransformRegistrar.class) +public class KinesisReadTransformRegistrar implements ExternalTransformRegistrar { Review comment: I think it would be cleaner just to have a single registrar with inner classes for all the builders and the configuration classes. WDYT? ########## File path: sdks/java/io/kinesis/expansion-service/build.gradle ########## @@ -0,0 +1,37 @@ +/* + * 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. + */ + +apply plugin: 'org.apache.beam.module' +apply plugin: 'application' +mainClassName = "org.apache.beam.sdk.expansion.service.ExpansionService" + +applyJavaNature(enableChecker:false, Review comment: Are you able to run the nullability checker, or does it complain about underlying issues in `:sdks:java:io:kinesis`? We should prefer enabling the nullability checker in new packages if possible since there's an effort to enable the checker throughout Beam (BEAM-10402). ########## File path: sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisIO.java ########## @@ -561,6 +569,41 @@ public Read withMaxCapacityPerShard(Integer maxCapacity) { } } + /** + * A {@link PTransform} to read from Kinesis stream. Similar to {@link KinesisIO.Read}, but + * removes Kinesis metatdata and returns a {@link PCollection} of {@link byte[]}. See {@link + * KinesisIO} for more information on usage and configuration of reader. + */ + public static class TypedWithoutMetadata extends PTransform<PBegin, PCollection<byte[]>> { Review comment: Also bikesheddy comment: I'm not sure about the name TypedWithoutMetadata, should it be ReadWithoutMetadata? ########## File path: sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisIO.java ########## @@ -561,6 +569,41 @@ public Read withMaxCapacityPerShard(Integer maxCapacity) { } } + /** + * A {@link PTransform} to read from Kinesis stream. Similar to {@link KinesisIO.Read}, but + * removes Kinesis metatdata and returns a {@link PCollection} of {@link byte[]}. See {@link + * KinesisIO} for more information on usage and configuration of reader. + */ + public static class TypedWithoutMetadata extends PTransform<PBegin, PCollection<byte[]>> { Review comment: We should file a jira for making `KinesisIO.Read` (with metadata) available. I think this would be possible if we register a schema for `KinesisRecord`, but it won't work cross-language until we have portable support for datetimes (i.e. BEAM-7554 will be a blocker). Also maybe this one should use a URN like "read_without_metadata" or "read_data_only" so we can use "read" in the future. ########## File path: sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisIO.java ########## @@ -536,6 +537,13 @@ public Read withMaxCapacityPerShard(Integer maxCapacity) { return toBuilder().setMaxCapacityPerShard(maxCapacity).build(); } + /** + * Returns a {@link PTransform} for PCollection of {@link byte[]}, dropping Kinesis metatdata. + */ + public PTransform<PBegin, PCollection<byte[]>> withoutMetadata() { + return new TypedWithoutMetadata(this); + } Review comment: I think it will be confusing to have this function here, since every other `with...` method returns a `Read` class. A user might expect to be able to do: ``` KinesisIO.read() .withoutMetadata() .withMaxCapacityPerShard(x) ``` Instead we should add a `KinesisIO.readWithoutMetadata()` that returns something like `TypedWithoutMetadata` with all the same configuration parameters as `Read`. You might take a look at how this is handled in `PubsubIO` for inspiration, I think it has a similar problem where it supports multiple different output types, but we want them all to share some configuration parameters. It looks like the way it's handled there is with a Read<T> type, and you pass in a function for converting PubsubMessage to T. Alternatively, you could move the logic to extract the payload into the external transform builder, rather than adding the interface here (I think there's value in adding it here though if you want to take it on). ---------------------------------------------------------------- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: [email protected]
