apilloud commented on a change in pull request #15915:
URL: https://github.com/apache/beam/pull/15915#discussion_r746898199
##########
File path:
sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/SqlTransform.java
##########
@@ -298,6 +304,11 @@ public SqlTransform registerUdaf(String functionName,
Combine.CombineFn combineF
return toBuilder().setUdafDefinitions(newUdafs).build();
}
+ public SqlTransform withErrorsTransformer(
+ PTransform<PCollection<BeamCalcRelError>, POutput> errorsTransformer) {
Review comment:
Can you make this `PTransform<PCollection<BeamCalcRelError>, ? extends
POutput>` instead? (Or really `PTransform<PCollection<Row>, ? extends POutput>`)
##########
File path:
sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamCalcRelError.java
##########
@@ -0,0 +1,34 @@
+/*
+ * 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.extensions.sql.impl.rel;
+
+import com.google.auto.value.AutoValue;
+import org.apache.beam.sdk.values.Row;
+
+@AutoValue
+public abstract class BeamCalcRelError {
+
+ public static BeamCalcRelError create(Row row, String error) {
Review comment:
It appears this class could just be replaced with `Row`?
```
schema = Schema.builder().addRowField("row",
row.getSchema()).addStringField("error").build()
Row.withSchema(schema).attachValues(row, error).build()
```
##########
File path:
sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamCalcRel.java
##########
@@ -192,9 +207,17 @@ public Calc copy(RelTraitSet traitSet, RelNode input,
RexProgram program) {
outputSchema,
options.getVerifyRowValues(),
getJarPaths(program),
- inputGetter.getFieldAccess());
-
- return upstream.apply(ParDo.of(calcFn)).setRowSchema(outputSchema);
+ inputGetter.getFieldAccess(),
+ errorsTransformer != null);
+
+ PCollectionTuple tuple =
+ upstream.apply(ParDo.of(calcFn).withOutputTags(rows,
TupleTagList.of(errors)));
+ PCollection<BeamCalcRelError> errorPCollection =
+
tuple.get(errors).setCoder(BeamCalcRelErrorCoder.of(RowCoder.of(upstream.getSchema())));
Review comment:
Thanks for the explanation, do keep this as Rows. Your schema is still
not quite right, I believe this is correct:
`RowCoder.of(SelectHelpers.getOutputSchema(upstream.getSchema(),
inputGetter.getFieldAccess()))`
You should add tests that cover more complex cases, like the graph above. It
would also be good to add a test that feeds multiple error transformers with
different rows from a single SqlTransform into a union PCollection with some
useful output (to replicate the typical use case of having a single dead letter
queue).
This works because we know the row at expand time and you have a different
error transformer per transform. If you went to the tagged output approach,
you'd need a tag per internal transform, so that seems like a non-starter.
--
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]