We read every piece of feedback, and take your input very seriously.
To see all available qualifiers, see our documentation.
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Caught by DataflowTemplates validation for Beam 2.61.0rc1: https://github.com/GoogleCloudPlatform/DataflowTemplates/pull/2014/checks?check_run_id=33057418339
Was able to bisect:
good: 20d0f6e
bad: c243491
To reproduce,
checkout Validate Beam 2.61.0rc1 GoogleCloudPlatform/DataflowTemplates#2014
on beam repo, after making any change, run
On Beam repo, after making any change (e.g. add log)
git cherry-pick c53a2017ba6e22816c8346fc8ce50142d30e3f81 ./gradlew :sdks:java:core:publishToMavenLocal -Ppublishing ./gradlew :runners:direct-java:publishToMavenLocal -Ppublishing
mvn test -pl v2/dataplex -Dtest=DataplexBigQueryToGcsTest#testE2E_mainPathWithAllStepsEnabled
Priority: 1 (data loss / total loss of function)
The text was updated successfully, but these errors were encountered:
A simple test: succeeded prior to the change, failing after the change:
package org.apache.beam.runners.direct; import com.google.auto.value.AutoValue; import java.io.Serializable; import org.apache.beam.sdk.coders.DefaultCoder; import org.apache.beam.sdk.schemas.AutoValueSchema; import org.apache.beam.sdk.schemas.SchemaCoder; import org.apache.beam.sdk.schemas.annotations.DefaultSchema; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.transforms.MapElements; import org.apache.beam.sdk.transforms.SerializableFunction; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.TypeDescriptor; import org.apache.beam.sdk.values.TypeDescriptors; import org.checkerframework.checker.nullness.qual.Nullable; import org.junit.Rule; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; @RunWith(JUnit4.class) public class NullableEvaluationTest { @Rule public TestPipeline p = TestPipeline.create(); @Test public void testConstruction() { // Either annotated with @Nullable or not does not matter p.apply(Create.of(KV.<String, @Nullable BigQueryTablePartition>of("123", null))) .apply( "MapFileNames", MapElements.into(TypeDescriptors.kvs( TypeDescriptor.of(BigQueryTablePartition.class), TypeDescriptor.of(String.class))) .via((SerializableFunction<KV<String, BigQueryTablePartition>, KV<BigQueryTablePartition, String>>) kv -> KV.of(kv.getValue(), kv.getKey()))); p.run(); } /** BigQuery table partition metadata. */ @AutoValue @DefaultCoder(SchemaCoder.class) @DefaultSchema(AutoValueSchema.class) public abstract static class BigQueryTablePartition implements Serializable { public abstract String getPartitionName(); public static Builder builder() { return new AutoValue_NullableEvaluationTest_BigQueryTablePartition.Builder(); } /** Builder for {@link BigQueryTablePartition}. */ @AutoValue.Builder public abstract static class Builder { public abstract Builder setPartitionName(String value); public abstract BigQueryTablePartition build(); } } }
Sorry, something went wrong.
No branches or pull requests
What happened?
Caught by DataflowTemplates validation for Beam 2.61.0rc1: https://github.com/GoogleCloudPlatform/DataflowTemplates/pull/2014/checks?check_run_id=33057418339
Was able to bisect:
good: 20d0f6e
bad: c243491
To reproduce,
checkout Validate Beam 2.61.0rc1 GoogleCloudPlatform/DataflowTemplates#2014
on beam repo, after making any change, run
On Beam repo, after making any change (e.g. add log)
mvn test -pl v2/dataplex -Dtest=DataplexBigQueryToGcsTest#testE2E_mainPathWithAllStepsEnabled
Issue Priority
Priority: 1 (data loss / total loss of function)
Issue Components
The text was updated successfully, but these errors were encountered: