ahmedabu98 commented on code in PR #27254: URL: https://github.com/apache/beam/pull/27254#discussion_r1243997044
########## sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BeamSchemaToBytesTransformers.java: ########## @@ -0,0 +1,115 @@ +/* + * 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 java.nio.charset.StandardCharsets; +import java.util.Objects; +import java.util.function.BiFunction; +import java.util.function.Function; +import java.util.stream.Collectors; +import javax.annotation.Nullable; +import org.apache.beam.sdk.schemas.Schema; +import org.apache.beam.sdk.values.Row; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.primitives.Bytes; +import org.checkerframework.checker.initialization.qual.UnknownInitialization; +import org.joda.time.format.ISODateTimeFormat; + +@SuppressWarnings("nullness") +public class BeamSchemaToBytesTransformers { + + @UnknownInitialization + @FunctionalInterface + public interface MyBiFunction<T, X, K> extends BiFunction<T, X, K> { + + @Override + @Nullable + K apply(T t, X u); + + @Override + default <V> BiFunction<T, X, V> andThen(Function<? super K, ? extends V> after) { + Objects.requireNonNull(after); + return (T t, X u) -> after.apply(apply(t, u)); + } + } + + private static byte[] toBytes(@Nullable Object obj) { + if (obj == null) { + return new byte[0]; + } else { + return obj.toString().getBytes(StandardCharsets.UTF_8); + } + } + + public static byte[] encodeArrayField(Row row, Schema.Field field) { + if (field.getType().getCollectionElementType().equals(Schema.FieldType.DATETIME)) { + throw new UnsupportedOperationException("Datetime arrays are not supported!"); + } else { + return Bytes.concat( + new byte[] {(byte) '['}, + row.getArray(field.getName()).stream() + .map(obj -> obj == null ? "" : obj.toString()) + .collect(Collectors.joining(",")) + .getBytes(StandardCharsets.UTF_8), + new byte[] {(byte) ']'}); + } + } + + @SuppressWarnings("DoNotCallSuggester") + public static byte[] encodeRowField(Row row, Schema.Field field) { + throw new UnsupportedOperationException("Nested fields are not supported!"); + } + + public static MyBiFunction<Row, Schema.Field, byte[]> getBytesEncoders(Schema.TypeName type) { + switch (type) { + case BYTES: + return ((row, field) -> row.getBytes(field.getName())); + case STRING: + return ((row, field) -> toBytes(row.getString(field.getName()))); + case BOOLEAN: + return ((row, field) -> toBytes(row.getBoolean(field.getName()))); + case DOUBLE: + return ((row, field) -> toBytes(row.getDouble(field.getName()))); + case FLOAT: + return ((row, field) -> toBytes(row.getFloat(field.getName()))); + case INT16: + return ((row, field) -> toBytes(row.getInt16(field.getName()))); + case INT32: + return ((row, field) -> toBytes(row.getInt32(field.getName()))); + case INT64: + return ((row, field) -> toBytes(row.getInt64(field.getName()))); + case DECIMAL: + return ((row, field) -> toBytes(row.getDecimal(field.getName()))); + case ARRAY: + return BeamSchemaToBytesTransformers::encodeArrayField; + case ROW: + return BeamSchemaToBytesTransformers::encodeRowField; + case DATETIME: + return ((row, field) -> { + if (row.getDateTime(field.getName()) == null) { + return new byte[0]; + } else { + return ISODateTimeFormat.basicDateTime() + .print(row.getDateTime(field.getName())) + .getBytes(StandardCharsets.UTF_8); + } + }); + default: + return null; Review Comment: why not throw the illegal argument exception here? would make BigTableIOWriteSchemaBasedTransform(lines 239-250) more readable ########## sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigTableIOWriteSchemaBasedTransform.java: ########## @@ -0,0 +1,322 @@ +/* + * 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 com.google.api.gax.rpc.AlreadyExistsException; +import com.google.bigtable.v2.Mutation; +import com.google.cloud.bigtable.admin.v2.BigtableTableAdminClient; +import com.google.cloud.bigtable.admin.v2.BigtableTableAdminSettings; +import com.google.cloud.bigtable.admin.v2.models.CreateTableRequest; +import com.google.cloud.bigtable.admin.v2.models.Table; +import com.google.cloud.bigtable.config.BigtableOptions; +import com.google.protobuf.ByteString; +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.util.List; +import java.util.Set; +import java.util.stream.Collectors; +import javax.annotation.Nullable; +import org.apache.beam.sdk.coders.ByteArrayCoder; +import org.apache.beam.sdk.coders.KvCoder; +import org.apache.beam.sdk.schemas.FieldAccessDescriptor; +import org.apache.beam.sdk.schemas.Schema; +import org.apache.beam.sdk.schemas.SchemaCoder; +import org.apache.beam.sdk.schemas.transforms.SchemaTransform; +import org.apache.beam.sdk.schemas.utils.RowSelector; +import org.apache.beam.sdk.schemas.utils.SelectHelpers; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.DoFn.OutputReceiver; +import org.apache.beam.sdk.transforms.DoFn.ProcessElement; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.transforms.WithKeys; +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.vendor.guava.v26_0_jre.com.google.common.base.Splitter; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Iterables; +import org.joda.time.Instant; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class BigTableIOWriteSchemaBasedTransform + extends PTransform<PCollectionRowTuple, PCollectionRowTuple> implements SchemaTransform { + + private static final Logger LOG = + LoggerFactory.getLogger(BigTableIOWriteSchemaBasedTransform.class); + public static final String INPUT_TAG = "input"; + + private final String projectId; + private final String instanceId; + private final String tableId; + @Nullable private final String bigTableEndpoint; + private final List<String> keyColumns; + private final Instant timestampForRows; + @Nullable private final String appProfileId; + + BigTableIOWriteSchemaBasedTransform( + String projectId, + String instanceId, + String tableId, + List<String> keyColumns, + @Nullable String bigTableEndpoint, + @Nullable String appProfileId) { + this.projectId = projectId; + this.instanceId = instanceId; + this.tableId = tableId; + this.keyColumns = keyColumns; + this.bigTableEndpoint = bigTableEndpoint; + this.timestampForRows = Instant.now(); + this.appProfileId = appProfileId; + } + + @Override + public PTransform<PCollectionRowTuple, PCollectionRowTuple> buildTransform() { + return this; + } + + public BigtableTableAdminClient bigtableTableAdminClient() throws IOException { + BigtableTableAdminSettings.Builder settingsBuilder; + if (this.bigTableEndpoint != null && !this.bigTableEndpoint.isEmpty()) { + settingsBuilder = + BigtableTableAdminSettings.newBuilderForEmulator( + Integer.parseInt(Iterables.get(Splitter.on(':').split(bigTableEndpoint), 1))) + .setInstanceId(instanceId) + .setProjectId(projectId); + settingsBuilder.stubSettings().setEndpoint(bigTableEndpoint); + } else { + settingsBuilder = + BigtableTableAdminSettings.newBuilder().setInstanceId(instanceId).setProjectId(projectId); + } + return BigtableTableAdminClient.create(settingsBuilder.build()); + } + + private void createTableIfNeeded(Schema inputSchema) { + try (BigtableTableAdminClient client = bigtableTableAdminClient()) { + CreateTableRequest createTableRequest = CreateTableRequest.of(tableId); + inputSchema.getFields().forEach(field -> createTableRequest.addFamily(field.getName())); + client.createTable(createTableRequest); + } catch (IOException e) { + throw new RuntimeException( + String.format( + "Failed to access BigTable instance %s in project %s", instanceId, projectId), + e); + } catch (io.grpc.StatusRuntimeException | com.google.api.gax.rpc.AlreadyExistsException e) { + if ((e.getMessage() != null + && e.getMessage().toLowerCase().contains("already") + && e.getMessage().toLowerCase().contains("exists")) + || e instanceof AlreadyExistsException) { + // The table already exists. We do not need to handle this. + LOG.info("Bigtable destination table {} already exists. Not creating a new one.", tableId); + } else { + throw new RuntimeException( + String.format( + "Unable to create BigTable table in instance %s in project %s", + instanceId, projectId), + e); + } + } + } + + private void verifyTableSchemaMatches(Schema inputSchema) { + // TODO(pabloem): What happens if we don't have privileges to create the table? + try (BigtableTableAdminClient client = bigtableTableAdminClient()) { + Table table = client.getTable(tableId); + Set<String> columnFamilies = + table.getColumnFamilies().stream().map(cf -> cf.getId()).collect(Collectors.toSet()); + Set<String> inputColumns = + inputSchema.getFields().stream() + .map(field -> field.getName()) + .collect(Collectors.toSet()); + + // All columns in the input must exist in BigTable, and they must be the same size + // TODO(pabloem): Do we support cases where BigTable column families is a SUPERSET of BQ + // columns? + // TODO(pabloem): Add a test case for this. + if (!(columnFamilies.containsAll(inputColumns) + && columnFamilies.size() == inputSchema.getFields().size())) { + throw new IllegalArgumentException( + String.format( + "Unable to match input schema with the columns of the destination " + + "table in Bigtable. Fields missing in BigTable: %s.", + inputColumns.removeAll(columnFamilies))); + } + } catch (IOException e) { + // TODO(pabloem): HANDLE THIS POSSIBILITY + } + } + + @Override + public PCollectionRowTuple expand(PCollectionRowTuple input) { + PCollection<Row> inputData = input.get(INPUT_TAG); + + inputData + .getSchema() + .getFields() + .forEach( + f -> { + if (f.getType().getTypeName().equals(Schema.TypeName.ROW)) { + throw new UnsupportedOperationException( + String.format( + "Nested fields are not supported. Field %s is of type ROW.", f.getName())); + } + }); + + Set<String> inputFields = + inputData.getSchema().getFields().stream() + .map(field -> field.getName()) + .collect(Collectors.toSet()); + if (!inputFields.containsAll(keyColumns)) { + throw new IllegalArgumentException( + String.format( + "Key columns selected were %s, however input schema only contains columns %s", + keyColumns, inputFields)); + } + + createTableIfNeeded(inputData.getSchema()); + verifyTableSchemaMatches(inputData.getSchema()); + + // STEP 1: Select the key columns from the input Rows + final Schema keySchema = + Schema.builder() + .addFields( + keyColumns.stream() + .map(colName -> inputData.getSchema().getField(colName)) + .collect(Collectors.toList())) + .build(); + + RowSelector keySelector = + new SelectHelpers.RowSelectorContainer( + inputData.getSchema(), + FieldAccessDescriptor.withFieldNames(keyColumns).resolve(inputData.getSchema()), + false); + + PCollection<KV<Row, Row>> keyedRows = + inputData + .apply(WithKeys.of(row -> keySelector.select(row))) + .setCoder(KvCoder.of(SchemaCoder.of(keySchema), SchemaCoder.of(inputData.getSchema()))); + + // STEP 2: Convert all data types to ByteString data + final Schema recordBytesSchema = + Schema.builder() + .addFields( + inputData.getSchema().getFields().stream() + .map(field -> Schema.Field.of(field.getName(), Schema.FieldType.BYTES)) + .collect(Collectors.toList())) + .build(); + + PCollection<KV<byte[], Row>> byteEncodedKeyedRows = + keyedRows + .apply( + "encodeKeys", + ParDo.of( + new DoFn<KV<Row, Row>, KV<byte[], Row>>() { + @SuppressWarnings("nullness") + @ProcessElement + public void process( + @DoFn.Element KV<Row, Row> elm, + OutputReceiver<KV<byte[], Row>> receiver) { + List<byte[]> byteEncodedColumns = + elm.getValue().getSchema().getFields().stream() + .map( + field -> { + if (BeamSchemaToBytesTransformers.getBytesEncoders( + field.getType().getTypeName()) + != null) { + return BeamSchemaToBytesTransformers.getBytesEncoders( + field.getType().getTypeName()) + .apply(elm.getValue(), field); + } else { + throw new IllegalArgumentException( + "Unsupported column type: " + + field.getType().getTypeName().toString()); + } + }) + .collect(Collectors.toList()); + + byte[] byteEncodedKeyCols = + StandardCharsets.UTF_8 + .encode( + String.join( + "", + elm.getKey().getValues().stream() + .map(columnValue -> columnValue.toString()) + .collect(Collectors.toList()))) + .array(); + + Row.Builder valueRow = Row.withSchema(recordBytesSchema); + // TODO(pabloem): This is more inefficient than valueRow.addValues(bEC), but + // that was giving + // me trouble so I didn't use it. + byteEncodedColumns.forEach(bytes -> valueRow.addValue(bytes)); + receiver.output(KV.of(byteEncodedKeyCols, valueRow.build())); + } + })) + .setCoder(KvCoder.of(ByteArrayCoder.of(), SchemaCoder.of(recordBytesSchema))); + + // STEP 3: Convert KV<bytes, Row> into KV<ByteString, List<SetCell<...>>> + PCollection<KV<ByteString, Iterable<Mutation>>> bigtableMutations = + byteEncodedKeyedRows.apply( + "buildMutations", + ParDo.of( + new DoFn<KV<byte[], Row>, KV<ByteString, Iterable<Mutation>>>() { + @SuppressWarnings("nullness") + @ProcessElement + public void process( + @DoFn.Element KV<byte[], Row> elm, + OutputReceiver<KV<ByteString, Iterable<Mutation>>> receiver) { + receiver.output( + KV.of( + ByteString.copyFrom(elm.getKey()), + elm.getValue().getSchema().getFields().stream() + .map( + field -> + Mutation.newBuilder() + .setSetCell( + Mutation.SetCell.newBuilder() + .setFamilyName(field.getName()) + .setTimestampMicros( + timestampForRows.getMillis() * 1000) Review Comment: What's the purpose of this timestamp? It's probably more accurate to instantiate a new Instant inline. Another approach is specifying `-1`, which will use Bigtable server time at ingestion. ########## sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigTableWriteSchemaTransformConfiguration.java: ########## @@ -0,0 +1,81 @@ +/* + * 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 com.google.auto.value.AutoValue; +import java.util.List; +import javax.annotation.Nullable; +import org.apache.beam.sdk.schemas.AutoValueSchema; +import org.apache.beam.sdk.schemas.annotations.DefaultSchema; +import org.apache.beam.sdk.schemas.annotations.SchemaFieldDescription; +import org.apache.beam.sdk.transforms.SerializableFunction; +import org.apache.beam.sdk.values.Row; +import org.apache.beam.sdk.values.TypeDescriptor; + +@DefaultSchema(AutoValueSchema.class) +@AutoValue +public abstract class BigTableWriteSchemaTransformConfiguration { + @SchemaFieldDescription("The Google Cloud project that the Bigtable instance is in.") + public abstract String getProjectId(); + + @SchemaFieldDescription("The ID of the Bigtable instance to write to.") + public abstract String getInstanceId(); + + @SchemaFieldDescription("The ID of the Bigtable table to write to.") + public abstract String getTableId(); + + @SchemaFieldDescription("Columns that make up a row's key.") + public abstract List<String> getKeyColumns(); + + public abstract @Nullable String getEndpoint(); + + @SchemaFieldDescription("The ID of the app profile used to connect to Bigtable.") + public abstract @Nullable String getAppProfileId(); + + public static Builder builder() { + return new AutoValue_BigTableWriteSchemaTransformConfiguration.Builder(); + } + + private static final AutoValueSchema AUTO_VALUE_SCHEMA = new AutoValueSchema(); + private static final TypeDescriptor<BigTableWriteSchemaTransformConfiguration> TYPE_DESCRIPTOR = + TypeDescriptor.of(BigTableWriteSchemaTransformConfiguration.class); + private static final SerializableFunction<BigTableWriteSchemaTransformConfiguration, Row> + ROW_SERIALIZABLE_FUNCTION = AUTO_VALUE_SCHEMA.toRowFunction(TYPE_DESCRIPTOR); + + /** Transform configuration to a {@link Row}. */ + public Row toBeamRow() { + return ROW_SERIALIZABLE_FUNCTION.apply(this); + } + + @AutoValue.Builder + public abstract static class Builder { + public abstract Builder setProjectId(String value); + + public abstract Builder setInstanceId(String value); + + public abstract Builder setTableId(String value); + + public abstract Builder setKeyColumns(List<String> value); + + public abstract Builder setEndpoint(@Nullable String endpoint); + + public abstract Builder setAppProfileId(@Nullable String appProfile); Review Comment: It's good practice to set identical value names (ie. projectId, instanceId, etc) ########## sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigTableWriteSchemaTransformProvider.java: ########## @@ -0,0 +1,61 @@ +/* + * 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 com.google.auto.service.AutoService; +import java.util.Collections; +import java.util.List; +import org.apache.beam.sdk.schemas.transforms.SchemaTransform; +import org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider; +import org.apache.beam.sdk.schemas.transforms.TypedSchemaTransformProvider; + +@AutoService(SchemaTransformProvider.class) +public class BigTableWriteSchemaTransformProvider + extends TypedSchemaTransformProvider<BigTableWriteSchemaTransformConfiguration> { + + @Override + public String identifier() { + return "beam:schematransform:org.apache.beam:bigtable_write:v1"; Review Comment: There's a BigtableWriteSchemaTransformProvider (for xlang) getting ready to be merged in #27146 with the same URN. ########## sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigTableWriteSchemaTransformConfiguration.java: ########## @@ -0,0 +1,81 @@ +/* + * 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 com.google.auto.value.AutoValue; +import java.util.List; +import javax.annotation.Nullable; +import org.apache.beam.sdk.schemas.AutoValueSchema; +import org.apache.beam.sdk.schemas.annotations.DefaultSchema; +import org.apache.beam.sdk.schemas.annotations.SchemaFieldDescription; +import org.apache.beam.sdk.transforms.SerializableFunction; +import org.apache.beam.sdk.values.Row; +import org.apache.beam.sdk.values.TypeDescriptor; + +@DefaultSchema(AutoValueSchema.class) +@AutoValue +public abstract class BigTableWriteSchemaTransformConfiguration { + @SchemaFieldDescription("The Google Cloud project that the Bigtable instance is in.") + public abstract String getProjectId(); + + @SchemaFieldDescription("The ID of the Bigtable instance to write to.") + public abstract String getInstanceId(); + + @SchemaFieldDescription("The ID of the Bigtable table to write to.") + public abstract String getTableId(); + + @SchemaFieldDescription("Columns that make up a row's key.") + public abstract List<String> getKeyColumns(); + + public abstract @Nullable String getEndpoint(); Review Comment: Missing a `@SchemaFieldDescription` here? ########## sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigTableWriteSchemaTransformConfiguration.java: ########## @@ -0,0 +1,81 @@ +/* + * 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 com.google.auto.value.AutoValue; +import java.util.List; +import javax.annotation.Nullable; +import org.apache.beam.sdk.schemas.AutoValueSchema; +import org.apache.beam.sdk.schemas.annotations.DefaultSchema; +import org.apache.beam.sdk.schemas.annotations.SchemaFieldDescription; +import org.apache.beam.sdk.transforms.SerializableFunction; +import org.apache.beam.sdk.values.Row; +import org.apache.beam.sdk.values.TypeDescriptor; + +@DefaultSchema(AutoValueSchema.class) +@AutoValue +public abstract class BigTableWriteSchemaTransformConfiguration { + @SchemaFieldDescription("The Google Cloud project that the Bigtable instance is in.") + public abstract String getProjectId(); + + @SchemaFieldDescription("The ID of the Bigtable instance to write to.") + public abstract String getInstanceId(); + + @SchemaFieldDescription("The ID of the Bigtable table to write to.") + public abstract String getTableId(); + + @SchemaFieldDescription("Columns that make up a row's key.") + public abstract List<String> getKeyColumns(); + + public abstract @Nullable String getEndpoint(); + + @SchemaFieldDescription("The ID of the app profile used to connect to Bigtable.") + public abstract @Nullable String getAppProfileId(); + + public static Builder builder() { + return new AutoValue_BigTableWriteSchemaTransformConfiguration.Builder(); + } + + private static final AutoValueSchema AUTO_VALUE_SCHEMA = new AutoValueSchema(); + private static final TypeDescriptor<BigTableWriteSchemaTransformConfiguration> TYPE_DESCRIPTOR = + TypeDescriptor.of(BigTableWriteSchemaTransformConfiguration.class); + private static final SerializableFunction<BigTableWriteSchemaTransformConfiguration, Row> + ROW_SERIALIZABLE_FUNCTION = AUTO_VALUE_SCHEMA.toRowFunction(TYPE_DESCRIPTOR); + + /** Transform configuration to a {@link Row}. */ + public Row toBeamRow() { + return ROW_SERIALIZABLE_FUNCTION.apply(this); + } Review Comment: I'm not sure if you need all of this ########## sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigTableIOWriteSchemaBasedTransform.java: ########## @@ -0,0 +1,322 @@ +/* + * 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 com.google.api.gax.rpc.AlreadyExistsException; +import com.google.bigtable.v2.Mutation; +import com.google.cloud.bigtable.admin.v2.BigtableTableAdminClient; +import com.google.cloud.bigtable.admin.v2.BigtableTableAdminSettings; +import com.google.cloud.bigtable.admin.v2.models.CreateTableRequest; +import com.google.cloud.bigtable.admin.v2.models.Table; +import com.google.cloud.bigtable.config.BigtableOptions; +import com.google.protobuf.ByteString; +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.util.List; +import java.util.Set; +import java.util.stream.Collectors; +import javax.annotation.Nullable; +import org.apache.beam.sdk.coders.ByteArrayCoder; +import org.apache.beam.sdk.coders.KvCoder; +import org.apache.beam.sdk.schemas.FieldAccessDescriptor; +import org.apache.beam.sdk.schemas.Schema; +import org.apache.beam.sdk.schemas.SchemaCoder; +import org.apache.beam.sdk.schemas.transforms.SchemaTransform; +import org.apache.beam.sdk.schemas.utils.RowSelector; +import org.apache.beam.sdk.schemas.utils.SelectHelpers; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.DoFn.OutputReceiver; +import org.apache.beam.sdk.transforms.DoFn.ProcessElement; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.transforms.WithKeys; +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.vendor.guava.v26_0_jre.com.google.common.base.Splitter; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Iterables; +import org.joda.time.Instant; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class BigTableIOWriteSchemaBasedTransform + extends PTransform<PCollectionRowTuple, PCollectionRowTuple> implements SchemaTransform { + + private static final Logger LOG = + LoggerFactory.getLogger(BigTableIOWriteSchemaBasedTransform.class); + public static final String INPUT_TAG = "input"; + + private final String projectId; + private final String instanceId; + private final String tableId; + @Nullable private final String bigTableEndpoint; + private final List<String> keyColumns; + private final Instant timestampForRows; + @Nullable private final String appProfileId; + + BigTableIOWriteSchemaBasedTransform( + String projectId, + String instanceId, + String tableId, + List<String> keyColumns, + @Nullable String bigTableEndpoint, + @Nullable String appProfileId) { + this.projectId = projectId; + this.instanceId = instanceId; + this.tableId = tableId; + this.keyColumns = keyColumns; + this.bigTableEndpoint = bigTableEndpoint; + this.timestampForRows = Instant.now(); + this.appProfileId = appProfileId; + } + + @Override + public PTransform<PCollectionRowTuple, PCollectionRowTuple> buildTransform() { + return this; + } + + public BigtableTableAdminClient bigtableTableAdminClient() throws IOException { + BigtableTableAdminSettings.Builder settingsBuilder; + if (this.bigTableEndpoint != null && !this.bigTableEndpoint.isEmpty()) { + settingsBuilder = + BigtableTableAdminSettings.newBuilderForEmulator( + Integer.parseInt(Iterables.get(Splitter.on(':').split(bigTableEndpoint), 1))) + .setInstanceId(instanceId) + .setProjectId(projectId); + settingsBuilder.stubSettings().setEndpoint(bigTableEndpoint); + } else { + settingsBuilder = + BigtableTableAdminSettings.newBuilder().setInstanceId(instanceId).setProjectId(projectId); + } + return BigtableTableAdminClient.create(settingsBuilder.build()); + } + + private void createTableIfNeeded(Schema inputSchema) { + try (BigtableTableAdminClient client = bigtableTableAdminClient()) { + CreateTableRequest createTableRequest = CreateTableRequest.of(tableId); + inputSchema.getFields().forEach(field -> createTableRequest.addFamily(field.getName())); + client.createTable(createTableRequest); + } catch (IOException e) { + throw new RuntimeException( + String.format( + "Failed to access BigTable instance %s in project %s", instanceId, projectId), + e); + } catch (io.grpc.StatusRuntimeException | com.google.api.gax.rpc.AlreadyExistsException e) { + if ((e.getMessage() != null + && e.getMessage().toLowerCase().contains("already") + && e.getMessage().toLowerCase().contains("exists")) + || e instanceof AlreadyExistsException) { + // The table already exists. We do not need to handle this. + LOG.info("Bigtable destination table {} already exists. Not creating a new one.", tableId); + } else { + throw new RuntimeException( + String.format( + "Unable to create BigTable table in instance %s in project %s", + instanceId, projectId), + e); + } + } + } + + private void verifyTableSchemaMatches(Schema inputSchema) { + // TODO(pabloem): What happens if we don't have privileges to create the table? + try (BigtableTableAdminClient client = bigtableTableAdminClient()) { + Table table = client.getTable(tableId); + Set<String> columnFamilies = + table.getColumnFamilies().stream().map(cf -> cf.getId()).collect(Collectors.toSet()); + Set<String> inputColumns = + inputSchema.getFields().stream() + .map(field -> field.getName()) + .collect(Collectors.toSet()); + + // All columns in the input must exist in BigTable, and they must be the same size Review Comment: ```suggestion // All column families in the input must exist in BigTable, and they must be the same size ``` ########## sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigTableIOWriteSchemaBasedTransform.java: ########## @@ -0,0 +1,322 @@ +/* + * 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 com.google.api.gax.rpc.AlreadyExistsException; +import com.google.bigtable.v2.Mutation; +import com.google.cloud.bigtable.admin.v2.BigtableTableAdminClient; +import com.google.cloud.bigtable.admin.v2.BigtableTableAdminSettings; +import com.google.cloud.bigtable.admin.v2.models.CreateTableRequest; +import com.google.cloud.bigtable.admin.v2.models.Table; +import com.google.cloud.bigtable.config.BigtableOptions; +import com.google.protobuf.ByteString; +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.util.List; +import java.util.Set; +import java.util.stream.Collectors; +import javax.annotation.Nullable; +import org.apache.beam.sdk.coders.ByteArrayCoder; +import org.apache.beam.sdk.coders.KvCoder; +import org.apache.beam.sdk.schemas.FieldAccessDescriptor; +import org.apache.beam.sdk.schemas.Schema; +import org.apache.beam.sdk.schemas.SchemaCoder; +import org.apache.beam.sdk.schemas.transforms.SchemaTransform; +import org.apache.beam.sdk.schemas.utils.RowSelector; +import org.apache.beam.sdk.schemas.utils.SelectHelpers; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.DoFn.OutputReceiver; +import org.apache.beam.sdk.transforms.DoFn.ProcessElement; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.transforms.WithKeys; +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.vendor.guava.v26_0_jre.com.google.common.base.Splitter; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Iterables; +import org.joda.time.Instant; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class BigTableIOWriteSchemaBasedTransform + extends PTransform<PCollectionRowTuple, PCollectionRowTuple> implements SchemaTransform { + + private static final Logger LOG = + LoggerFactory.getLogger(BigTableIOWriteSchemaBasedTransform.class); + public static final String INPUT_TAG = "input"; + + private final String projectId; + private final String instanceId; + private final String tableId; + @Nullable private final String bigTableEndpoint; + private final List<String> keyColumns; + private final Instant timestampForRows; + @Nullable private final String appProfileId; + + BigTableIOWriteSchemaBasedTransform( + String projectId, + String instanceId, + String tableId, + List<String> keyColumns, + @Nullable String bigTableEndpoint, + @Nullable String appProfileId) { + this.projectId = projectId; + this.instanceId = instanceId; + this.tableId = tableId; + this.keyColumns = keyColumns; + this.bigTableEndpoint = bigTableEndpoint; + this.timestampForRows = Instant.now(); + this.appProfileId = appProfileId; + } + + @Override + public PTransform<PCollectionRowTuple, PCollectionRowTuple> buildTransform() { + return this; + } + + public BigtableTableAdminClient bigtableTableAdminClient() throws IOException { + BigtableTableAdminSettings.Builder settingsBuilder; + if (this.bigTableEndpoint != null && !this.bigTableEndpoint.isEmpty()) { + settingsBuilder = + BigtableTableAdminSettings.newBuilderForEmulator( + Integer.parseInt(Iterables.get(Splitter.on(':').split(bigTableEndpoint), 1))) + .setInstanceId(instanceId) + .setProjectId(projectId); + settingsBuilder.stubSettings().setEndpoint(bigTableEndpoint); + } else { + settingsBuilder = + BigtableTableAdminSettings.newBuilder().setInstanceId(instanceId).setProjectId(projectId); + } + return BigtableTableAdminClient.create(settingsBuilder.build()); + } + + private void createTableIfNeeded(Schema inputSchema) { + try (BigtableTableAdminClient client = bigtableTableAdminClient()) { + CreateTableRequest createTableRequest = CreateTableRequest.of(tableId); + inputSchema.getFields().forEach(field -> createTableRequest.addFamily(field.getName())); + client.createTable(createTableRequest); + } catch (IOException e) { + throw new RuntimeException( + String.format( + "Failed to access BigTable instance %s in project %s", instanceId, projectId), + e); + } catch (io.grpc.StatusRuntimeException | com.google.api.gax.rpc.AlreadyExistsException e) { + if ((e.getMessage() != null + && e.getMessage().toLowerCase().contains("already") + && e.getMessage().toLowerCase().contains("exists")) + || e instanceof AlreadyExistsException) { + // The table already exists. We do not need to handle this. + LOG.info("Bigtable destination table {} already exists. Not creating a new one.", tableId); + } else { + throw new RuntimeException( + String.format( + "Unable to create BigTable table in instance %s in project %s", + instanceId, projectId), + e); + } + } + } + + private void verifyTableSchemaMatches(Schema inputSchema) { + // TODO(pabloem): What happens if we don't have privileges to create the table? + try (BigtableTableAdminClient client = bigtableTableAdminClient()) { + Table table = client.getTable(tableId); + Set<String> columnFamilies = + table.getColumnFamilies().stream().map(cf -> cf.getId()).collect(Collectors.toSet()); + Set<String> inputColumns = + inputSchema.getFields().stream() + .map(field -> field.getName()) + .collect(Collectors.toSet()); + + // All columns in the input must exist in BigTable, and they must be the same size + // TODO(pabloem): Do we support cases where BigTable column families is a SUPERSET of BQ + // columns? + // TODO(pabloem): Add a test case for this. + if (!(columnFamilies.containsAll(inputColumns) + && columnFamilies.size() == inputSchema.getFields().size())) { + throw new IllegalArgumentException( + String.format( + "Unable to match input schema with the columns of the destination " + + "table in Bigtable. Fields missing in BigTable: %s.", + inputColumns.removeAll(columnFamilies))); + } + } catch (IOException e) { + // TODO(pabloem): HANDLE THIS POSSIBILITY + } + } + + @Override + public PCollectionRowTuple expand(PCollectionRowTuple input) { + PCollection<Row> inputData = input.get(INPUT_TAG); + + inputData + .getSchema() + .getFields() + .forEach( + f -> { + if (f.getType().getTypeName().equals(Schema.TypeName.ROW)) { + throw new UnsupportedOperationException( + String.format( + "Nested fields are not supported. Field %s is of type ROW.", f.getName())); + } + }); + + Set<String> inputFields = + inputData.getSchema().getFields().stream() + .map(field -> field.getName()) + .collect(Collectors.toSet()); + if (!inputFields.containsAll(keyColumns)) { + throw new IllegalArgumentException( + String.format( + "Key columns selected were %s, however input schema only contains columns %s", + keyColumns, inputFields)); + } + + createTableIfNeeded(inputData.getSchema()); + verifyTableSchemaMatches(inputData.getSchema()); + + // STEP 1: Select the key columns from the input Rows + final Schema keySchema = + Schema.builder() + .addFields( + keyColumns.stream() + .map(colName -> inputData.getSchema().getField(colName)) + .collect(Collectors.toList())) + .build(); + + RowSelector keySelector = + new SelectHelpers.RowSelectorContainer( + inputData.getSchema(), + FieldAccessDescriptor.withFieldNames(keyColumns).resolve(inputData.getSchema()), + false); + + PCollection<KV<Row, Row>> keyedRows = + inputData + .apply(WithKeys.of(row -> keySelector.select(row))) + .setCoder(KvCoder.of(SchemaCoder.of(keySchema), SchemaCoder.of(inputData.getSchema()))); + + // STEP 2: Convert all data types to ByteString data + final Schema recordBytesSchema = + Schema.builder() + .addFields( + inputData.getSchema().getFields().stream() + .map(field -> Schema.Field.of(field.getName(), Schema.FieldType.BYTES)) + .collect(Collectors.toList())) + .build(); + + PCollection<KV<byte[], Row>> byteEncodedKeyedRows = + keyedRows + .apply( + "encodeKeys", + ParDo.of( + new DoFn<KV<Row, Row>, KV<byte[], Row>>() { + @SuppressWarnings("nullness") Review Comment: Like @johnjcasey is saying, we're trying to remove these annotations from the beam repo -- 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: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
