ahmedabu98 commented on code in PR #35435: URL: https://github.com/apache/beam/pull/35435#discussion_r2197875878
########## sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableSimpleWriteSchemaTransformProvider.java: ########## @@ -0,0 +1,211 @@ +/* + * 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.gcp.bigtable; + +import static java.util.Optional.ofNullable; +import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument; + +import com.google.auto.service.AutoService; +import com.google.bigtable.v2.Mutation; +import com.google.bigtable.v2.TimestampRange; +import com.google.protobuf.ByteString; +import java.util.Objects; +import org.apache.beam.sdk.io.gcp.bigtable.BigtableWriteSchemaTransformProvider.BigtableWriteSchemaTransformConfiguration; +import org.apache.beam.sdk.schemas.transforms.SchemaTransform; +import org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider; +import org.apache.beam.sdk.schemas.transforms.TypedSchemaTransformProvider; +import org.apache.beam.sdk.transforms.GroupByKey; +import org.apache.beam.sdk.transforms.MapElements; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PCollectionRowTuple; +import org.apache.beam.sdk.values.Row; +import org.apache.beam.sdk.values.TypeDescriptor; +import org.apache.beam.sdk.values.TypeDescriptors; + +/** + * An implementation of {@link TypedSchemaTransformProvider} for Bigtable Write jobs configured via + * {@link BigtableWriteSchemaTransformConfiguration}. + */ +@AutoService(SchemaTransformProvider.class) +public class BigtableSimpleWriteSchemaTransformProvider Review Comment: suggestion: change to `BigtableMutationWriteSchemaTransform`/`Provider` to more accurately describe what this transform is doing (users can use it to write individual mutations) ########## sdks/python/apache_beam/yaml/integration_tests.py: ########## @@ -39,21 +41,38 @@ from testcontainers.core.container import DockerContainer from testcontainers.core.waiting_utils import wait_for_logs from testcontainers.google import PubSubContainer -from testcontainers.kafka import KafkaContainer +# from testcontainers.kafka import KafkaContainer Review Comment: nit: uncomment some of these lines before merging ########## sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableSimpleWriteSchemaTransformProvider.java: ########## @@ -0,0 +1,211 @@ +/* + * 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.gcp.bigtable; + +import static java.util.Optional.ofNullable; +import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument; + +import com.google.auto.service.AutoService; +import com.google.bigtable.v2.Mutation; +import com.google.bigtable.v2.TimestampRange; +import com.google.protobuf.ByteString; +import java.util.Objects; +import org.apache.beam.sdk.io.gcp.bigtable.BigtableWriteSchemaTransformProvider.BigtableWriteSchemaTransformConfiguration; +import org.apache.beam.sdk.schemas.transforms.SchemaTransform; +import org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider; +import org.apache.beam.sdk.schemas.transforms.TypedSchemaTransformProvider; +import org.apache.beam.sdk.transforms.GroupByKey; +import org.apache.beam.sdk.transforms.MapElements; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PCollectionRowTuple; +import org.apache.beam.sdk.values.Row; +import org.apache.beam.sdk.values.TypeDescriptor; +import org.apache.beam.sdk.values.TypeDescriptors; + +/** + * An implementation of {@link TypedSchemaTransformProvider} for Bigtable Write jobs configured via + * {@link BigtableWriteSchemaTransformConfiguration}. + */ +@AutoService(SchemaTransformProvider.class) +public class BigtableSimpleWriteSchemaTransformProvider + extends TypedSchemaTransformProvider<BigtableWriteSchemaTransformConfiguration> { + + private static final String INPUT_TAG = "input"; + + @Override + protected SchemaTransform from(BigtableWriteSchemaTransformConfiguration configuration) { + return new BigtableSimpleWriteSchemaTransform(configuration); + } + + @Override + public String identifier() { + return "beam:schematransform:org.apache.beam:bigtable_simple_write:v1"; + } + + /** + * A {@link SchemaTransform} for Bigtable writes, configured with {@link + * BigtableWriteSchemaTransformConfiguration} and instantiated by {@link + * BigtableWriteSchemaTransformProvider}. + */ + private static class BigtableSimpleWriteSchemaTransform extends SchemaTransform { + private final BigtableWriteSchemaTransformConfiguration configuration; + + BigtableSimpleWriteSchemaTransform(BigtableWriteSchemaTransformConfiguration configuration) { + configuration.validate(); + this.configuration = configuration; + } + + @Override + public PCollectionRowTuple expand(PCollectionRowTuple input) { + checkArgument( + input.has(INPUT_TAG), + String.format( + "Could not find expected input [%s] to %s.", INPUT_TAG, getClass().getSimpleName())); + + PCollection<KV<ByteString, Iterable<Mutation>>> bigtableMutations = + changeMutationInput(input); + + bigtableMutations.apply( + BigtableIO.write() + .withTableId(configuration.getTableId()) + .withInstanceId(configuration.getInstanceId()) + .withProjectId(configuration.getProjectId())); + + return PCollectionRowTuple.empty(input.getPipeline()); + } + + public PCollection<KV<ByteString, Iterable<Mutation>>> changeMutationInput( + PCollectionRowTuple inputR) { + PCollection<Row> beamRowMutationsList = inputR.getSinglePCollection(); + // convert all row inputs into KV<ByteString, Mutation> + PCollection<KV<ByteString, Mutation>> changedBeamRowMutationsList = + beamRowMutationsList.apply( + MapElements.into( + TypeDescriptors.kvs( + TypeDescriptor.of(ByteString.class), TypeDescriptor.of(Mutation.class))) + .via( + (Row input) -> { + @SuppressWarnings("nullness") + ByteString key = + ByteString.copyFrom(((Objects.requireNonNull(input.getBytes("key"))))); + + Mutation bigtableMutation; + String mutationType = + input.getString("type"); // Direct call, can return null + if (mutationType == null) { + throw new IllegalArgumentException("Mutation type cannot be null."); + } + switch (mutationType) { + case "SetCell": + @SuppressWarnings("nullness") + Mutation.SetCell.Builder setMutation = + Mutation.SetCell.newBuilder() + .setValue( + ByteString.copyFrom( + ((Objects.requireNonNull(input.getBytes("value")))))) Review Comment: small nit: throughout this switch-case block, I recommend using `org.apache.beam.sdk.util.Preconditions.checkStateNotNull()` instead of `java.util.Objects.requireNonNull()` ########## sdks/python/apache_beam/yaml/tests/bigTable.yaml: ########## @@ -0,0 +1,124 @@ +# +# 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. +# + + +fixtures: + - name: BT_TABLE + type: "apache_beam.yaml.integration_tests.temp_bigtable_table" + config: + project: "apache-beam-testing" + - name: TEMP_DIR + # Need distributed filesystem to be able to read and write from a container. + type: "apache_beam.yaml.integration_tests.gcs_temp_dir" + config: + bucket: "gs://temp-storage-for-end-to-end-tests/temp-it" + + # Tests for BigTable YAML IO + +pipelines: + - pipeline: + type: chain + transforms: + - type: Create + config: + elements: + - {key: 'row1', Review Comment: nit: i think yaml will recognize that this is a dict without needing to include the curly braces ########## sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableSimpleWriteSchemaTransformProvider.java: ########## @@ -0,0 +1,211 @@ +/* + * 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.gcp.bigtable; + +import static java.util.Optional.ofNullable; +import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument; + +import com.google.auto.service.AutoService; +import com.google.bigtable.v2.Mutation; +import com.google.bigtable.v2.TimestampRange; +import com.google.protobuf.ByteString; +import java.util.Objects; +import org.apache.beam.sdk.io.gcp.bigtable.BigtableWriteSchemaTransformProvider.BigtableWriteSchemaTransformConfiguration; +import org.apache.beam.sdk.schemas.transforms.SchemaTransform; +import org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider; +import org.apache.beam.sdk.schemas.transforms.TypedSchemaTransformProvider; +import org.apache.beam.sdk.transforms.GroupByKey; +import org.apache.beam.sdk.transforms.MapElements; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PCollectionRowTuple; +import org.apache.beam.sdk.values.Row; +import org.apache.beam.sdk.values.TypeDescriptor; +import org.apache.beam.sdk.values.TypeDescriptors; + +/** + * An implementation of {@link TypedSchemaTransformProvider} for Bigtable Write jobs configured via + * {@link BigtableWriteSchemaTransformConfiguration}. + */ +@AutoService(SchemaTransformProvider.class) +public class BigtableSimpleWriteSchemaTransformProvider + extends TypedSchemaTransformProvider<BigtableWriteSchemaTransformConfiguration> { + + private static final String INPUT_TAG = "input"; + + @Override + protected SchemaTransform from(BigtableWriteSchemaTransformConfiguration configuration) { + return new BigtableSimpleWriteSchemaTransform(configuration); + } + + @Override + public String identifier() { + return "beam:schematransform:org.apache.beam:bigtable_simple_write:v1"; + } + + /** + * A {@link SchemaTransform} for Bigtable writes, configured with {@link + * BigtableWriteSchemaTransformConfiguration} and instantiated by {@link + * BigtableWriteSchemaTransformProvider}. + */ + private static class BigtableSimpleWriteSchemaTransform extends SchemaTransform { + private final BigtableWriteSchemaTransformConfiguration configuration; + + BigtableSimpleWriteSchemaTransform(BigtableWriteSchemaTransformConfiguration configuration) { + configuration.validate(); + this.configuration = configuration; + } + + @Override + public PCollectionRowTuple expand(PCollectionRowTuple input) { + checkArgument( Review Comment: I think it'd be good to validate the input schema early. We know what it should look like to fit this use-case, so if the input schema is invalid, we should fail with a helpful message ########## sdks/python/apache_beam/yaml/standard_io.yaml: ########## @@ -371,3 +371,27 @@ 'WriteToTFRecord': 'beam:schematransform:org.apache.beam:tfrecord_write:v1' config: gradle_target: 'sdks:java:io:expansion-service:shadowJar' + +#BigTable +- type: renaming + transforms: + 'ReadFromBigTable': 'ReadFromBigTable' + 'WriteToBigTable': 'WriteToBigTable' + config: + mappings: + 'ReadFromBigTable': + project: 'project_Id' + instance: 'instanceId' + table: 'tableId' + 'WriteToBigTable': + project: 'project_id' + instance: 'instance_id' + table: 'table_id' + Rows: "rows" Review Comment: what is the `Rows` config parameter for? ########## sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableSimpleWriteSchemaTransformProvider.java: ########## @@ -0,0 +1,211 @@ +/* + * 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.gcp.bigtable; + +import static java.util.Optional.ofNullable; +import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument; + +import com.google.auto.service.AutoService; +import com.google.bigtable.v2.Mutation; +import com.google.bigtable.v2.TimestampRange; +import com.google.protobuf.ByteString; +import java.util.Objects; +import org.apache.beam.sdk.io.gcp.bigtable.BigtableWriteSchemaTransformProvider.BigtableWriteSchemaTransformConfiguration; +import org.apache.beam.sdk.schemas.transforms.SchemaTransform; +import org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider; +import org.apache.beam.sdk.schemas.transforms.TypedSchemaTransformProvider; +import org.apache.beam.sdk.transforms.GroupByKey; +import org.apache.beam.sdk.transforms.MapElements; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PCollectionRowTuple; +import org.apache.beam.sdk.values.Row; +import org.apache.beam.sdk.values.TypeDescriptor; +import org.apache.beam.sdk.values.TypeDescriptors; + +/** + * An implementation of {@link TypedSchemaTransformProvider} for Bigtable Write jobs configured via + * {@link BigtableWriteSchemaTransformConfiguration}. + */ +@AutoService(SchemaTransformProvider.class) +public class BigtableSimpleWriteSchemaTransformProvider + extends TypedSchemaTransformProvider<BigtableWriteSchemaTransformConfiguration> { + + private static final String INPUT_TAG = "input"; + + @Override + protected SchemaTransform from(BigtableWriteSchemaTransformConfiguration configuration) { + return new BigtableSimpleWriteSchemaTransform(configuration); + } + + @Override + public String identifier() { + return "beam:schematransform:org.apache.beam:bigtable_simple_write:v1"; + } + + /** + * A {@link SchemaTransform} for Bigtable writes, configured with {@link + * BigtableWriteSchemaTransformConfiguration} and instantiated by {@link + * BigtableWriteSchemaTransformProvider}. + */ + private static class BigtableSimpleWriteSchemaTransform extends SchemaTransform { + private final BigtableWriteSchemaTransformConfiguration configuration; + + BigtableSimpleWriteSchemaTransform(BigtableWriteSchemaTransformConfiguration configuration) { + configuration.validate(); + this.configuration = configuration; + } + + @Override + public PCollectionRowTuple expand(PCollectionRowTuple input) { + checkArgument( + input.has(INPUT_TAG), + String.format( + "Could not find expected input [%s] to %s.", INPUT_TAG, getClass().getSimpleName())); + + PCollection<KV<ByteString, Iterable<Mutation>>> bigtableMutations = + changeMutationInput(input); + + bigtableMutations.apply( + BigtableIO.write() + .withTableId(configuration.getTableId()) + .withInstanceId(configuration.getInstanceId()) + .withProjectId(configuration.getProjectId())); + + return PCollectionRowTuple.empty(input.getPipeline()); + } + + public PCollection<KV<ByteString, Iterable<Mutation>>> changeMutationInput( + PCollectionRowTuple inputR) { + PCollection<Row> beamRowMutationsList = inputR.getSinglePCollection(); + // convert all row inputs into KV<ByteString, Mutation> + PCollection<KV<ByteString, Mutation>> changedBeamRowMutationsList = + beamRowMutationsList.apply( + MapElements.into( + TypeDescriptors.kvs( + TypeDescriptor.of(ByteString.class), TypeDescriptor.of(Mutation.class))) + .via( + (Row input) -> { + @SuppressWarnings("nullness") + ByteString key = + ByteString.copyFrom(((Objects.requireNonNull(input.getBytes("key"))))); + + Mutation bigtableMutation; + String mutationType = + input.getString("type"); // Direct call, can return null + if (mutationType == null) { + throw new IllegalArgumentException("Mutation type cannot be null."); Review Comment: We have a helpful `org.apache.beam.sdk.util.Preconditions.checkStateNotNull()` that helps with these nullable variables. e.g. you can do `String mutationType = Preconditions.checkStateNotNull(input.getString("type"), "Mutation type cannot be null");` and it will give you a non-null String ########## sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableSimpleWriteSchemaTransformProvider.java: ########## @@ -0,0 +1,211 @@ +/* + * 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.gcp.bigtable; + +import static java.util.Optional.ofNullable; +import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument; + +import com.google.auto.service.AutoService; +import com.google.bigtable.v2.Mutation; +import com.google.bigtable.v2.TimestampRange; +import com.google.protobuf.ByteString; +import java.util.Objects; +import org.apache.beam.sdk.io.gcp.bigtable.BigtableWriteSchemaTransformProvider.BigtableWriteSchemaTransformConfiguration; +import org.apache.beam.sdk.schemas.transforms.SchemaTransform; +import org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider; +import org.apache.beam.sdk.schemas.transforms.TypedSchemaTransformProvider; +import org.apache.beam.sdk.transforms.GroupByKey; +import org.apache.beam.sdk.transforms.MapElements; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PCollectionRowTuple; +import org.apache.beam.sdk.values.Row; +import org.apache.beam.sdk.values.TypeDescriptor; +import org.apache.beam.sdk.values.TypeDescriptors; + +/** + * An implementation of {@link TypedSchemaTransformProvider} for Bigtable Write jobs configured via + * {@link BigtableWriteSchemaTransformConfiguration}. + */ +@AutoService(SchemaTransformProvider.class) +public class BigtableSimpleWriteSchemaTransformProvider + extends TypedSchemaTransformProvider<BigtableWriteSchemaTransformConfiguration> { + + private static final String INPUT_TAG = "input"; + + @Override + protected SchemaTransform from(BigtableWriteSchemaTransformConfiguration configuration) { + return new BigtableSimpleWriteSchemaTransform(configuration); + } + + @Override + public String identifier() { + return "beam:schematransform:org.apache.beam:bigtable_simple_write:v1"; + } + + /** + * A {@link SchemaTransform} for Bigtable writes, configured with {@link + * BigtableWriteSchemaTransformConfiguration} and instantiated by {@link + * BigtableWriteSchemaTransformProvider}. + */ + private static class BigtableSimpleWriteSchemaTransform extends SchemaTransform { + private final BigtableWriteSchemaTransformConfiguration configuration; + + BigtableSimpleWriteSchemaTransform(BigtableWriteSchemaTransformConfiguration configuration) { + configuration.validate(); + this.configuration = configuration; + } + + @Override + public PCollectionRowTuple expand(PCollectionRowTuple input) { + checkArgument( + input.has(INPUT_TAG), + String.format( + "Could not find expected input [%s] to %s.", INPUT_TAG, getClass().getSimpleName())); + + PCollection<KV<ByteString, Iterable<Mutation>>> bigtableMutations = + changeMutationInput(input); + + bigtableMutations.apply( + BigtableIO.write() + .withTableId(configuration.getTableId()) + .withInstanceId(configuration.getInstanceId()) + .withProjectId(configuration.getProjectId())); + + return PCollectionRowTuple.empty(input.getPipeline()); + } + + public PCollection<KV<ByteString, Iterable<Mutation>>> changeMutationInput( + PCollectionRowTuple inputR) { + PCollection<Row> beamRowMutationsList = inputR.getSinglePCollection(); + // convert all row inputs into KV<ByteString, Mutation> + PCollection<KV<ByteString, Mutation>> changedBeamRowMutationsList = + beamRowMutationsList.apply( + MapElements.into( + TypeDescriptors.kvs( + TypeDescriptor.of(ByteString.class), TypeDescriptor.of(Mutation.class))) + .via( + (Row input) -> { + @SuppressWarnings("nullness") + ByteString key = + ByteString.copyFrom(((Objects.requireNonNull(input.getBytes("key"))))); + + Mutation bigtableMutation; + String mutationType = + input.getString("type"); // Direct call, can return null + if (mutationType == null) { + throw new IllegalArgumentException("Mutation type cannot be null."); + } + switch (mutationType) { + case "SetCell": + @SuppressWarnings("nullness") + Mutation.SetCell.Builder setMutation = + Mutation.SetCell.newBuilder() + .setValue( + ByteString.copyFrom( + ((Objects.requireNonNull(input.getBytes("value")))))) Review Comment: plus including a helpful error message like "encountered SetCell mutation with missing 'value' property" -- 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. To unsubscribe, e-mail: github-unsubscr...@beam.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org