Skip to content

[Bug]: ClassCastException when using custom DynamicDestination in BigQueryIO.Write #22543

@dartiga

Description

@dartiga

What happened?

Beam: 2.40

While using custom DynamicDestination in BigQueryIO.Write got the following exception:

java.lang.ClassCastException: org.apache.beam.sdk.io.gcp.bigquery.TableDestination cannot be cast to java.lang.String com.king.da.destinations.KingAppDestinations.getTable(KingAppDestinations.java:17)
 org.apache.beam.sdk.io.gcp.bigquery.UpdateSchemaDestination.processElement(UpdateSchemaDestination.java:131)
   org.apache.beam.sdk.io.gcp.bigquery.UpdateSchemaDestination$DoFnInvoker.invokeProcessElement(Unknown Source)

Find below test case to reproduce:

import java.io.IOException;
import java.security.SecureRandom;
import java.util.Arrays;

import org.apache.beam.sdk.Pipeline;
import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO;
import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.CreateDisposition;
import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.WriteDisposition;
import org.apache.beam.sdk.io.gcp.bigquery.DynamicDestinations;
import org.apache.beam.sdk.io.gcp.bigquery.TableDestination;
import org.apache.beam.sdk.io.gcp.bigquery.TestBigQueryOptions;
import org.apache.beam.sdk.io.gcp.testing.BigqueryClient;
import org.apache.beam.sdk.testing.TestPipeline;
import org.apache.beam.sdk.transforms.Create;
import org.apache.beam.sdk.values.KV;
import org.apache.beam.sdk.values.PCollection;
import org.apache.beam.sdk.values.ValueInSingleWindow;
import org.checkerframework.checker.nullness.qual.Nullable;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.runner.RunWith;
import org.junit.runners.JUnit4;

import com.google.api.services.bigquery.model.TableFieldSchema;
import com.google.api.services.bigquery.model.TableRow;
import com.google.api.services.bigquery.model.TableSchema;

@RunWith(JUnit4.class)
public class UpdateSchemaDestinationTest {

    static final BigqueryClient BQ_CLIENT = new BigqueryClient(UpdateSchemaDestinationTest.class.getName());

    static final String DATASET_ID =
            "schema_update_options_class_cast_excepption"
                + System.currentTimeMillis()
                + "_"
                + new SecureRandom().nextInt(32);

    static TestBigQueryOptions options = TestPipeline.testingPipelineOptions().as(TestBigQueryOptions.class);

    static Pipeline pipeline;

    @BeforeClass
    public static void setUpAll() throws IOException, InterruptedException {
        options.setTempLocation(options.getTempRoot() + "/bq_it_temp");

        pipeline = Pipeline.create(options);

        BQ_CLIENT.createNewDataset(options.getProject(), DATASET_ID);
    }

    @AfterClass
    public static void tearDownAll() {
        BQ_CLIENT.deleteDataset(options.getProject(), DATASET_ID);
    }

    @Test
    public void classCastExceptionRegression() {
        DynamicDestinations<KV<String, String>, String> destinations = new SomeDynamicDestinations();

        PCollection<KV<String, String>> rows = pipeline
            .apply(Create.of(
                KV.of("table","foo"),
                KV.of("table","bar")
            ));

        rows.apply(BigQueryIO.<KV<String, String>>write()
            .withFormatFunction(kv -> new TableRow().set("name", kv.getValue()))
            .withCreateDisposition(CreateDisposition.CREATE_IF_NEEDED)
            .withWriteDisposition(WriteDisposition.WRITE_APPEND)
            .withMaxBytesPerPartition(1)
            .to(destinations));

        pipeline.run().waitUntilFinish();
    }

    private static final class SomeDynamicDestinations extends DynamicDestinations<KV<String, String>, String> {

        private static final long serialVersionUID = 1L;

        @Override
        public String getDestination(@Nullable ValueInSingleWindow<KV<String, String>> element) {
            return element.getValue().getKey();
        }

        @Override
        public TableDestination getTable(String destination) {
            return new TableDestination(DATASET_ID + "." + destination, "a table");
        }

        @Override
        public @Nullable TableSchema getSchema(String destination) {
            return new TableSchema().setFields(Arrays.asList(new TableFieldSchema().setName("name").setType("STRING")));
        }
    }
}

Issue Priority

Priority: 1

Issue Component

Component: io-java-gcp

Metadata

Metadata

Type

No type

Projects

No projects

Milestone

No milestone

Relationships

None yet

Development

No branches or pull requests

Issue actions