ahmedabu98 commented on code in PR #35435:
URL: https://github.com/apache/beam/pull/35435#discussion_r2213605757
##########
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableWriteSchemaTransformProvider.java:
##########
@@ -135,18 +140,186 @@ public PCollectionRowTuple expand(PCollectionRowTuple
input) {
String.format(
"Could not find expected input [%s] to %s.", INPUT_TAG,
getClass().getSimpleName()));
- PCollection<Row> beamRowMutations = input.get(INPUT_TAG);
- PCollection<KV<ByteString, Iterable<Mutation>>> bigtableMutations =
- beamRowMutations.apply(MapElements.via(new
GetMutationsFromBeamRow()));
+ Schema testOriginialSchema =
+ Schema.builder()
+ .addByteArrayField("key")
+ .addArrayField(
+ "mutations",
+ Schema.FieldType.map(Schema.FieldType.STRING,
Schema.FieldType.BYTES))
+ .build();
- bigtableMutations.apply(
- BigtableIO.write()
- .withTableId(configuration.getTableId())
- .withInstanceId(configuration.getInstanceId())
- .withProjectId(configuration.getProjectId()));
+ Schema inputSchema = input.getSinglePCollection().getSchema();
+ System.out.println("Input Schema for BigTableMutations: " + inputSchema);
+
+ PCollection<KV<ByteString, Iterable<Mutation>>> bigtableMutations = null;
+ if (inputSchema.equals(testOriginialSchema)) {
+ PCollection<Row> beamRowMutations = input.get(INPUT_TAG);
+ bigtableMutations =
+ beamRowMutations.apply(
+ // Original schema inputs gets sent out to the original
transform provider mutations
+ // function
+ MapElements.via(
+ new
BigtableWriteSchemaTransformProvider.GetMutationsFromBeamRow()));
+ } else if (inputSchema.hasField("type")) {
+ bigtableMutations = changeMutationInput(input);
+ } else {
+ System.out.println(
+ "Inputted Schema is Invalid; the schema should be formatted in one
of two ways:\n "
+ + "key\": ByteString\n"
+ + "\"type\": String\n"
+ + "\"column_qualifier\": ByteString\n"
+ + "\"family_name\": ByteString\n"
+ + "\"timestamp_micros\": Long\n"
+ + "\"start_timestamp_micros\": Long\n"
+ + "\"end_timestamp_micros\": Long"
+ + "OR\n"
+ + "\n"
+ + "\"key\": ByteString\n"
+ + "(\"mutations\", contains map(String, ByteString) of
mutations in the mutation schema format");
+ }
+
+ if (bigtableMutations != null) {
+ bigtableMutations.apply(
+ BigtableIO.write()
+ .withTableId(configuration.getTableId())
+ .withInstanceId(configuration.getInstanceId())
+ .withProjectId(configuration.getProjectId()));
+ } else {
+ throw new RuntimeException(
+ "Inputted Schema caused mutation error, check error logs and input
schema format");
+ }
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")
Review Comment:
we shouldn't need this SuppressWarning because there is a
`Preconditions.checkStateNotNull`
##########
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableWriteSchemaTransformProvider.java:
##########
@@ -135,18 +140,197 @@ public PCollectionRowTuple expand(PCollectionRowTuple
input) {
String.format(
"Could not find expected input [%s] to %s.", INPUT_TAG,
getClass().getSimpleName()));
- PCollection<Row> beamRowMutations = input.get(INPUT_TAG);
- PCollection<KV<ByteString, Iterable<Mutation>>> bigtableMutations =
- beamRowMutations.apply(MapElements.via(new
GetMutationsFromBeamRow()));
+ Schema testOriginialSchema =
+ Schema.builder()
+ .addByteArrayField("key")
+ .addArrayField(
+ "mutations",
+ Schema.FieldType.map(Schema.FieldType.STRING,
Schema.FieldType.BYTES))
+ .build();
- bigtableMutations.apply(
- BigtableIO.write()
- .withTableId(configuration.getTableId())
- .withInstanceId(configuration.getInstanceId())
- .withProjectId(configuration.getProjectId()));
+ Schema inputSchema = input.getSinglePCollection().getSchema();
+ System.out.println("Input Schema for BigTableMutations: " + inputSchema);
Review Comment:
We should log the input schema only if it's relevant (e.g. if it's invalid
and causing failures). It's not very useful when everything is WAI (and
probably will just be noisy)
##########
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableWriteSchemaTransformProvider.java:
##########
@@ -135,18 +140,186 @@ public PCollectionRowTuple expand(PCollectionRowTuple
input) {
String.format(
"Could not find expected input [%s] to %s.", INPUT_TAG,
getClass().getSimpleName()));
- PCollection<Row> beamRowMutations = input.get(INPUT_TAG);
- PCollection<KV<ByteString, Iterable<Mutation>>> bigtableMutations =
- beamRowMutations.apply(MapElements.via(new
GetMutationsFromBeamRow()));
+ Schema testOriginialSchema =
+ Schema.builder()
+ .addByteArrayField("key")
+ .addArrayField(
+ "mutations",
+ Schema.FieldType.map(Schema.FieldType.STRING,
Schema.FieldType.BYTES))
+ .build();
- bigtableMutations.apply(
- BigtableIO.write()
- .withTableId(configuration.getTableId())
- .withInstanceId(configuration.getInstanceId())
- .withProjectId(configuration.getProjectId()));
+ Schema inputSchema = input.getSinglePCollection().getSchema();
+ System.out.println("Input Schema for BigTableMutations: " + inputSchema);
+
+ PCollection<KV<ByteString, Iterable<Mutation>>> bigtableMutations = null;
+ if (inputSchema.equals(testOriginialSchema)) {
+ PCollection<Row> beamRowMutations = input.get(INPUT_TAG);
+ bigtableMutations =
+ beamRowMutations.apply(
+ // Original schema inputs gets sent out to the original
transform provider mutations
+ // function
+ MapElements.via(
+ new
BigtableWriteSchemaTransformProvider.GetMutationsFromBeamRow()));
+ } else if (inputSchema.hasField("type")) {
+ bigtableMutations = changeMutationInput(input);
+ } else {
+ System.out.println(
+ "Inputted Schema is Invalid; the schema should be formatted in one
of two ways:\n "
+ + "key\": ByteString\n"
+ + "\"type\": String\n"
+ + "\"column_qualifier\": ByteString\n"
+ + "\"family_name\": ByteString\n"
+ + "\"timestamp_micros\": Long\n"
+ + "\"start_timestamp_micros\": Long\n"
+ + "\"end_timestamp_micros\": Long"
+ + "OR\n"
+ + "\n"
+ + "\"key\": ByteString\n"
+ + "(\"mutations\", contains map(String, ByteString) of
mutations in the mutation schema format");
+ }
+
+ if (bigtableMutations != null) {
+ bigtableMutations.apply(
+ BigtableIO.write()
+ .withTableId(configuration.getTableId())
+ .withInstanceId(configuration.getInstanceId())
+ .withProjectId(configuration.getProjectId()));
+ } else {
+ throw new RuntimeException(
+ "Inputted Schema caused mutation error, check error logs and input
schema format");
+ }
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(
+ Preconditions.checkStateNotNull(
+ input.getBytes("key"),
+ "Encountered row with incorrect 'key'
property."));
+
+ 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(
+ Preconditions.checkStateNotNull(
+ input.getBytes("value"),
+ "Encountered SetCell mutation
with incorrect 'family_name' property.")))
+ .setColumnQualifier(
+ ByteString.copyFrom(
+ Preconditions.checkStateNotNull(
+
input.getBytes("column_qualifier"),
+ "Encountered SetCell mutation
with incorrect 'column_qualifier' property. ")))
+ .setFamilyNameBytes(
+ ByteString.copyFrom(
+ Preconditions.checkStateNotNull(
+ input.getBytes("family_name"),
+ "Encountered SetCell mutation
with incorrect 'family_name' property.")));
+ // Use timestamp if provided, else default to -1
(current
+ // Bigtable
+ // server time)
+ // Timestamp (optional, assuming Long type in Row
schema)
+ Long timestampMicros =
input.getInt64("timestamp_micros");
+ setMutation.setTimestampMicros(
+ timestampMicros != null ? timestampMicros :
-1);
+
+ bigtableMutation =
+
Mutation.newBuilder().setSetCell(setMutation.build()).build();
+ break;
+ case "DeleteFromColumn":
+ // set timestamp range if applicable
+ @SuppressWarnings("nullness")
+ Mutation.DeleteFromColumn.Builder deleteMutation =
+ Mutation.DeleteFromColumn.newBuilder()
+ .setColumnQualifier(
+ ByteString.copyFrom(
+ Preconditions.checkStateNotNull(
+
input.getBytes("column_qualifier"),
+ "Encountered DeleteFromColumn
mutation with incorrect 'column_qualifier' property.")))
+ .setFamilyNameBytes(
+ ByteString.copyFrom(
+ Preconditions.checkStateNotNull(
+ input.getBytes("family_name"),
+ "Encountered DeleteFromColumn
mutation with incorrect 'family_name' property.")));
+
+ // if start or end timestamp provided
+ // Timestamp Range (optional, assuming Long type
in Row schema)
+ Long startTimestampMicros = null;
+ Long endTimestampMicros = null;
+
+ if
(input.getSchema().hasField("start_timestamp_micros")) {
+ startTimestampMicros =
input.getInt64("start_timestamp_micros");
+ }
+ if
(input.getSchema().hasField("end_timestamp_micros")) {
+ endTimestampMicros =
input.getInt64("end_timestamp_micros");
+ }
+
+ if (startTimestampMicros != null ||
endTimestampMicros != null) {
+ TimestampRange.Builder timeRange =
TimestampRange.newBuilder();
+ if (startTimestampMicros != null) {
+
timeRange.setStartTimestampMicros(startTimestampMicros);
+ }
+ if (endTimestampMicros != null) {
+
timeRange.setEndTimestampMicros(endTimestampMicros);
+ }
+ deleteMutation.setTimeRange(timeRange.build());
+ }
+ bigtableMutation =
+ Mutation.newBuilder()
+
.setDeleteFromColumn(deleteMutation.build())
+ .build();
+ break;
+ case "DeleteFromFamily":
+ bigtableMutation =
+ Mutation.newBuilder()
+ .setDeleteFromFamily(
+ Mutation.DeleteFromFamily.newBuilder()
+ .setFamilyNameBytes(
+ ByteString.copyFrom(
+
Preconditions.checkStateNotNull(
+
input.getBytes("family_name"),
+ "Encountered
DeleteFromFamily mutation with incorrect 'family_name' property.")))
+ .build())
+ .build();
+ break;
+ case "DeleteFromRow":
+ bigtableMutation =
+ Mutation.newBuilder()
+
.setDeleteFromRow(Mutation.DeleteFromRow.newBuilder().build())
+ .build();
+ break;
+ default:
+ throw new RuntimeException(
+ String.format(
+ "Unexpected mutation type [%s]: %s",
+ ((input.getString("type"))), input));
Review Comment:
```suggestion
"Unexpected mutation type '%s' for key:
%s",
mutationType, key));
```
Generally, we don't print the whole data record because it might contain
sensitive information. Printing the key should be enough to help debug.
##########
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableWriteSchemaTransformProvider.java:
##########
@@ -135,18 +140,186 @@ public PCollectionRowTuple expand(PCollectionRowTuple
input) {
String.format(
"Could not find expected input [%s] to %s.", INPUT_TAG,
getClass().getSimpleName()));
- PCollection<Row> beamRowMutations = input.get(INPUT_TAG);
- PCollection<KV<ByteString, Iterable<Mutation>>> bigtableMutations =
- beamRowMutations.apply(MapElements.via(new
GetMutationsFromBeamRow()));
+ Schema testOriginialSchema =
+ Schema.builder()
+ .addByteArrayField("key")
+ .addArrayField(
+ "mutations",
+ Schema.FieldType.map(Schema.FieldType.STRING,
Schema.FieldType.BYTES))
+ .build();
- bigtableMutations.apply(
- BigtableIO.write()
- .withTableId(configuration.getTableId())
- .withInstanceId(configuration.getInstanceId())
- .withProjectId(configuration.getProjectId()));
+ Schema inputSchema = input.getSinglePCollection().getSchema();
+ System.out.println("Input Schema for BigTableMutations: " + inputSchema);
+
+ PCollection<KV<ByteString, Iterable<Mutation>>> bigtableMutations = null;
+ if (inputSchema.equals(testOriginialSchema)) {
+ PCollection<Row> beamRowMutations = input.get(INPUT_TAG);
+ bigtableMutations =
+ beamRowMutations.apply(
+ // Original schema inputs gets sent out to the original
transform provider mutations
+ // function
+ MapElements.via(
+ new
BigtableWriteSchemaTransformProvider.GetMutationsFromBeamRow()));
Review Comment:
```suggestion
new GetMutationsFromBeamRow()));
```
##########
sdks/python/apache_beam/yaml/integration_tests.py:
##########
@@ -696,6 +754,8 @@ def parse_test_files(filepattern):
For example, 'path/to/tests/*.yaml'.
"""
for path in glob.glob(filepattern):
+ # get rid of this before PR
+ #if "bigTable" in path:
Review Comment:
nit: cleanup
##########
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableWriteSchemaTransformProvider.java:
##########
@@ -135,18 +140,186 @@ public PCollectionRowTuple expand(PCollectionRowTuple
input) {
String.format(
"Could not find expected input [%s] to %s.", INPUT_TAG,
getClass().getSimpleName()));
- PCollection<Row> beamRowMutations = input.get(INPUT_TAG);
- PCollection<KV<ByteString, Iterable<Mutation>>> bigtableMutations =
- beamRowMutations.apply(MapElements.via(new
GetMutationsFromBeamRow()));
+ Schema testOriginialSchema =
+ Schema.builder()
+ .addByteArrayField("key")
+ .addArrayField(
+ "mutations",
+ Schema.FieldType.map(Schema.FieldType.STRING,
Schema.FieldType.BYTES))
+ .build();
- bigtableMutations.apply(
- BigtableIO.write()
- .withTableId(configuration.getTableId())
- .withInstanceId(configuration.getInstanceId())
- .withProjectId(configuration.getProjectId()));
+ Schema inputSchema = input.getSinglePCollection().getSchema();
+ System.out.println("Input Schema for BigTableMutations: " + inputSchema);
+
+ PCollection<KV<ByteString, Iterable<Mutation>>> bigtableMutations = null;
+ if (inputSchema.equals(testOriginialSchema)) {
+ PCollection<Row> beamRowMutations = input.get(INPUT_TAG);
+ bigtableMutations =
+ beamRowMutations.apply(
+ // Original schema inputs gets sent out to the original
transform provider mutations
+ // function
+ MapElements.via(
+ new
BigtableWriteSchemaTransformProvider.GetMutationsFromBeamRow()));
+ } else if (inputSchema.hasField("type")) {
+ bigtableMutations = changeMutationInput(input);
+ } else {
+ System.out.println(
+ "Inputted Schema is Invalid; the schema should be formatted in one
of two ways:\n "
+ + "key\": ByteString\n"
+ + "\"type\": String\n"
+ + "\"column_qualifier\": ByteString\n"
+ + "\"family_name\": ByteString\n"
+ + "\"timestamp_micros\": Long\n"
+ + "\"start_timestamp_micros\": Long\n"
+ + "\"end_timestamp_micros\": Long"
+ + "OR\n"
+ + "\n"
+ + "\"key\": ByteString\n"
+ + "(\"mutations\", contains map(String, ByteString) of
mutations in the mutation schema format");
+ }
+
+ if (bigtableMutations != null) {
+ bigtableMutations.apply(
+ BigtableIO.write()
+ .withTableId(configuration.getTableId())
+ .withInstanceId(configuration.getInstanceId())
+ .withProjectId(configuration.getProjectId()));
+ } else {
+ throw new RuntimeException(
+ "Inputted Schema caused mutation error, check error logs and input
schema format");
+ }
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(
+ Preconditions.checkStateNotNull(
+ input.getBytes("key"),
+ "Encountered row with incorrect 'key'
property."));
+
+ 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(
+ Preconditions.checkStateNotNull(
+ input.getBytes("value"),
+ "Encountered SetCell mutation
with incorrect 'family_name' property.")))
Review Comment:
```suggestion
"Encountered SetCell
mutation with missing 'family_name' property.")))
```
same with the rest of the messages below
##########
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableWriteSchemaTransformProvider.java:
##########
@@ -135,18 +140,186 @@ public PCollectionRowTuple expand(PCollectionRowTuple
input) {
String.format(
"Could not find expected input [%s] to %s.", INPUT_TAG,
getClass().getSimpleName()));
- PCollection<Row> beamRowMutations = input.get(INPUT_TAG);
- PCollection<KV<ByteString, Iterable<Mutation>>> bigtableMutations =
- beamRowMutations.apply(MapElements.via(new
GetMutationsFromBeamRow()));
+ Schema testOriginialSchema =
+ Schema.builder()
+ .addByteArrayField("key")
+ .addArrayField(
+ "mutations",
+ Schema.FieldType.map(Schema.FieldType.STRING,
Schema.FieldType.BYTES))
+ .build();
- bigtableMutations.apply(
- BigtableIO.write()
- .withTableId(configuration.getTableId())
- .withInstanceId(configuration.getInstanceId())
- .withProjectId(configuration.getProjectId()));
+ Schema inputSchema = input.getSinglePCollection().getSchema();
+ System.out.println("Input Schema for BigTableMutations: " + inputSchema);
+
+ PCollection<KV<ByteString, Iterable<Mutation>>> bigtableMutations = null;
+ if (inputSchema.equals(testOriginialSchema)) {
+ PCollection<Row> beamRowMutations = input.get(INPUT_TAG);
+ bigtableMutations =
+ beamRowMutations.apply(
+ // Original schema inputs gets sent out to the original
transform provider mutations
+ // function
+ MapElements.via(
+ new
BigtableWriteSchemaTransformProvider.GetMutationsFromBeamRow()));
+ } else if (inputSchema.hasField("type")) {
+ bigtableMutations = changeMutationInput(input);
+ } else {
+ System.out.println(
+ "Inputted Schema is Invalid; the schema should be formatted in one
of two ways:\n "
+ + "key\": ByteString\n"
+ + "\"type\": String\n"
+ + "\"column_qualifier\": ByteString\n"
+ + "\"family_name\": ByteString\n"
+ + "\"timestamp_micros\": Long\n"
+ + "\"start_timestamp_micros\": Long\n"
+ + "\"end_timestamp_micros\": Long"
+ + "OR\n"
+ + "\n"
+ + "\"key\": ByteString\n"
+ + "(\"mutations\", contains map(String, ByteString) of
mutations in the mutation schema format");
+ }
+
+ if (bigtableMutations != null) {
+ bigtableMutations.apply(
+ BigtableIO.write()
+ .withTableId(configuration.getTableId())
+ .withInstanceId(configuration.getInstanceId())
+ .withProjectId(configuration.getProjectId()));
+ } else {
+ throw new RuntimeException(
+ "Inputted Schema caused mutation error, check error logs and input
schema format");
+ }
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")
Review Comment:
Same with the rest of the SuppressWarnings below. We should handle null
values instead of suppressing
##########
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableWriteSchemaTransformProvider.java:
##########
@@ -135,18 +140,197 @@ public PCollectionRowTuple expand(PCollectionRowTuple
input) {
String.format(
"Could not find expected input [%s] to %s.", INPUT_TAG,
getClass().getSimpleName()));
- PCollection<Row> beamRowMutations = input.get(INPUT_TAG);
- PCollection<KV<ByteString, Iterable<Mutation>>> bigtableMutations =
- beamRowMutations.apply(MapElements.via(new
GetMutationsFromBeamRow()));
+ Schema testOriginialSchema =
+ Schema.builder()
+ .addByteArrayField("key")
+ .addArrayField(
+ "mutations",
+ Schema.FieldType.map(Schema.FieldType.STRING,
Schema.FieldType.BYTES))
+ .build();
- bigtableMutations.apply(
- BigtableIO.write()
- .withTableId(configuration.getTableId())
- .withInstanceId(configuration.getInstanceId())
- .withProjectId(configuration.getProjectId()));
+ Schema inputSchema = input.getSinglePCollection().getSchema();
+ System.out.println("Input Schema for BigTableMutations: " + inputSchema);
+
+ PCollection<KV<ByteString, Iterable<Mutation>>> bigtableMutations = null;
+ if (inputSchema.equals(testOriginialSchema)) {
+ PCollection<Row> beamRowMutations = input.get(INPUT_TAG);
+ bigtableMutations =
+ beamRowMutations.apply(
+ // Original schema inputs gets sent out to the original
transform provider mutations
+ // function
+ MapElements.via(
+ new
BigtableWriteSchemaTransformProvider.GetMutationsFromBeamRow()));
+ } else if (inputSchema.hasField("type")) {
+ // // validate early doesn't work for all mutations IT test but
it does help
+ // if (inputSchema.hasField("column_qualifier")) {
+ // Schema.FieldType columnQualifierType =
+ // inputSchema.getField("column_qualifier").getType();
+ // checkState(
+ // columnQualifierType.equals(Schema.FieldType.STRING)
+ // ||
columnQualifierType.equals(Schema.FieldType.BYTES),
+ // "column_qualifier should be of type STRING or BYTES");
+ // }
+ // // new schema inputs get sent to the new transform provider
mutation function
+ bigtableMutations = changeMutationInput(input);
+ } else {
+ System.out.println(
Review Comment:
Just saw you throw a RuntimeException below. I think that can be thrown here
instead with the message you have here
##########
sdks/python/apache_beam/yaml/integration_tests.py:
##########
@@ -677,6 +734,7 @@ def test(self, providers=providers): # default arg to
capture loop value
yield f'test_{suffix}', test
+# Add bigTable, if not big table it skips (temporarily)
Review Comment:
nit: cleanup
##########
sdks/python/apache_beam/yaml/tests/bigTable.yaml:
##########
@@ -0,0 +1,122 @@
+#
Review Comment:
nit: can we name this file `bigtable.yaml`? We generally don't have
uppercase for yaml files
##########
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableWriteSchemaTransformProvider.java:
##########
@@ -135,18 +140,186 @@ public PCollectionRowTuple expand(PCollectionRowTuple
input) {
String.format(
"Could not find expected input [%s] to %s.", INPUT_TAG,
getClass().getSimpleName()));
- PCollection<Row> beamRowMutations = input.get(INPUT_TAG);
- PCollection<KV<ByteString, Iterable<Mutation>>> bigtableMutations =
- beamRowMutations.apply(MapElements.via(new
GetMutationsFromBeamRow()));
+ Schema testOriginialSchema =
+ Schema.builder()
+ .addByteArrayField("key")
+ .addArrayField(
+ "mutations",
+ Schema.FieldType.map(Schema.FieldType.STRING,
Schema.FieldType.BYTES))
+ .build();
Review Comment:
nit: this should be a static variable
nit: `testOriginialSchema` --> `BATCHED_MUTATIONS_SCHEMA` (static variable
name)
##########
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableWriteSchemaTransformProvider.java:
##########
@@ -135,18 +140,197 @@ public PCollectionRowTuple expand(PCollectionRowTuple
input) {
String.format(
"Could not find expected input [%s] to %s.", INPUT_TAG,
getClass().getSimpleName()));
- PCollection<Row> beamRowMutations = input.get(INPUT_TAG);
- PCollection<KV<ByteString, Iterable<Mutation>>> bigtableMutations =
- beamRowMutations.apply(MapElements.via(new
GetMutationsFromBeamRow()));
+ Schema testOriginialSchema =
+ Schema.builder()
+ .addByteArrayField("key")
+ .addArrayField(
+ "mutations",
+ Schema.FieldType.map(Schema.FieldType.STRING,
Schema.FieldType.BYTES))
+ .build();
- bigtableMutations.apply(
- BigtableIO.write()
- .withTableId(configuration.getTableId())
- .withInstanceId(configuration.getInstanceId())
- .withProjectId(configuration.getProjectId()));
+ Schema inputSchema = input.getSinglePCollection().getSchema();
+ System.out.println("Input Schema for BigTableMutations: " + inputSchema);
+
+ PCollection<KV<ByteString, Iterable<Mutation>>> bigtableMutations = null;
+ if (inputSchema.equals(testOriginialSchema)) {
+ PCollection<Row> beamRowMutations = input.get(INPUT_TAG);
+ bigtableMutations =
+ beamRowMutations.apply(
+ // Original schema inputs gets sent out to the original
transform provider mutations
+ // function
+ MapElements.via(
+ new
BigtableWriteSchemaTransformProvider.GetMutationsFromBeamRow()));
+ } else if (inputSchema.hasField("type")) {
+ // // validate early doesn't work for all mutations IT test but
it does help
+ // if (inputSchema.hasField("column_qualifier")) {
+ // Schema.FieldType columnQualifierType =
+ // inputSchema.getField("column_qualifier").getType();
+ // checkState(
+ // columnQualifierType.equals(Schema.FieldType.STRING)
+ // ||
columnQualifierType.equals(Schema.FieldType.BYTES),
+ // "column_qualifier should be of type STRING or BYTES");
+ // }
+ // // new schema inputs get sent to the new transform provider
mutation function
+ bigtableMutations = changeMutationInput(input);
+ } else {
+ System.out.println(
Review Comment:
This should be throwing an error right? not just printing?
--
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]