Skip to content
New issue

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

add setJoinSubsetType to inject joinSubsetType to BeamSqlSeekableTable #28477

Merged
merged 5 commits into from
Sep 20, 2023
Merged
Show file tree
Hide file tree
Changes from 2 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@
import java.io.Serializable;
import java.util.List;
import org.apache.beam.sdk.options.PipelineOptions;
import org.apache.beam.sdk.schemas.Schema;
import org.apache.beam.sdk.transforms.DoFn;
import org.apache.beam.sdk.values.Row;

Expand All @@ -28,8 +29,12 @@
* FROM FACT_TABLE JOIN LOOKUP_TABLE ON ...}.
*/
public interface BeamSqlSeekableTable extends Serializable {
/** prepare the instance. */
default void setUp() {}
/**
* prepare the instance
Abacn marked this conversation as resolved.
Show resolved Hide resolved
*
* @param joinSubsetType joining subset schema
*/
default void setUp(Schema joinSubsetType) {}
gabrywu marked this conversation as resolved.
Show resolved Hide resolved

default void startBundle(
DoFn<Row, Row>.StartBundleContext context, PipelineOptions pipelineOptions) {}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -153,7 +153,7 @@ public PCollection<Row> expand(PCollection<Row> input) {
new DoFn<Row, Row>() {
@Setup
public void setup() {
seekableTable.setUp();
seekableTable.setUp(joinSubsetType);
}

@StartBundle
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -34,6 +34,7 @@
import org.apache.beam.sdk.values.POutput;
import org.apache.beam.sdk.values.Row;
import org.hamcrest.core.StringContains;
import org.junit.Assert;
import org.junit.BeforeClass;
import org.junit.Rule;
import org.junit.Test;
Expand All @@ -47,11 +48,18 @@ public class BeamSideInputLookupJoinRelTest extends BaseRelTest {

/** Test table for JOIN-AS-LOOKUP. */
public static class SiteLookupTable extends SchemaBaseBeamTable implements BeamSqlSeekableTable {
private Schema joinSubsetType;

public SiteLookupTable(Schema schema) {
super(schema);
}

@Override
public void setUp(Schema joinSubsetType) {
this.joinSubsetType = joinSubsetType;
Assert.assertNotNull(joinSubsetType);
}

@Override
public PCollection.IsBounded isBounded() {
return PCollection.IsBounded.BOUNDED;
Expand All @@ -69,6 +77,7 @@ public POutput buildIOWriter(PCollection<Row> input) {

@Override
public List<Row> seekRow(Row lookupSubRow) {
Assert.assertEquals(joinSubsetType, lookupSubRow.getSchema());
if (lookupSubRow.getInt32("site_id") == 2) {
return Arrays.asList(Row.withSchema(getSchema()).addValues(2, "SITE1").build());
}
Expand Down
Loading