Skip to content
Merged
Show file tree
Hide file tree
Changes from all 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 @@ -76,6 +76,10 @@ message ManagedTransforms {
"beam:schematransform:org.apache.beam:bigquery_write:v1"];
ICEBERG_CDC_READ = 6 [(org.apache.beam.model.pipeline.v1.beam_urn) =
"beam:schematransform:org.apache.beam:iceberg_cdc_read:v1"];
POSTGRES_READ = 7 [(org.apache.beam.model.pipeline.v1.beam_urn) =
"beam:schematransform:org.apache.beam:postgres_read:v1"];
POSTGRES_WRITE = 8 [(org.apache.beam.model.pipeline.v1.beam_urn) =
"beam:schematransform:org.apache.beam:postgres_write:v1"];
}
}

Expand Down
3 changes: 3 additions & 0 deletions sdks/java/io/jdbc/build.gradle
Original file line number Diff line number Diff line change
Expand Up @@ -29,6 +29,7 @@ ext.summary = "IO to read and write on JDBC datasource."
dependencies {
implementation library.java.vendored_guava_32_1_2_jre
implementation project(path: ":sdks:java:core", configuration: "shadow")
implementation project(path: ":model:pipeline", configuration: "shadow")
implementation library.java.dbcp2
implementation library.java.joda_time
implementation "org.apache.commons:commons-pool2:2.11.1"
Expand All @@ -39,8 +40,10 @@ dependencies {
testImplementation project(path: ":sdks:java:core", configuration: "shadowTest")
testImplementation project(path: ":sdks:java:extensions:avro", configuration: "testRuntimeMigration")
testImplementation project(path: ":sdks:java:io:common")
testImplementation project(path: ":sdks:java:managed")
testImplementation project(path: ":sdks:java:testing:test-utils")
testImplementation library.java.junit
testImplementation library.java.mockito_inline
testImplementation library.java.slf4j_api
testImplementation library.java.postgres

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,8 @@
import java.util.Objects;
import javax.annotation.Nullable;
import org.apache.beam.sdk.schemas.AutoValueSchema;
import org.apache.beam.sdk.schemas.NoSuchSchemaException;
import org.apache.beam.sdk.schemas.SchemaRegistry;
import org.apache.beam.sdk.schemas.annotations.DefaultSchema;
import org.apache.beam.sdk.schemas.annotations.SchemaFieldDescription;
import org.apache.beam.sdk.schemas.transforms.SchemaTransform;
Expand Down Expand Up @@ -265,6 +267,20 @@ public PCollectionRowTuple expand(PCollectionRowTuple input) {
}
return PCollectionRowTuple.of("output", input.getPipeline().apply(readRows));
}

public Row getConfigurationRow() {
try {
// To stay consistent with our SchemaTransform configuration naming conventions,
// we sort lexicographically
return SchemaRegistry.createDefault()
.getToRowFunction(JdbcReadSchemaTransformConfiguration.class)
.apply(config)
.sorted()
.toSnakeCase();
} catch (NoSuchSchemaException e) {
throw new RuntimeException(e);
}
}
}

@Override
Expand Down Expand Up @@ -401,6 +417,8 @@ public static Builder builder() {
.Builder();
}

public abstract Builder toBuilder();

@AutoValue.Builder
public abstract static class Builder {
public abstract Builder setDriverClassName(String value);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -27,7 +27,9 @@
import java.util.Objects;
import javax.annotation.Nullable;
import org.apache.beam.sdk.schemas.AutoValueSchema;
import org.apache.beam.sdk.schemas.NoSuchSchemaException;
import org.apache.beam.sdk.schemas.Schema;
import org.apache.beam.sdk.schemas.SchemaRegistry;
import org.apache.beam.sdk.schemas.annotations.DefaultSchema;
import org.apache.beam.sdk.schemas.annotations.SchemaFieldDescription;
import org.apache.beam.sdk.schemas.transforms.SchemaTransform;
Expand Down Expand Up @@ -265,6 +267,20 @@ public PCollectionRowTuple expand(PCollectionRowTuple input) {
.setRowSchema(Schema.of());
return PCollectionRowTuple.of("post_write", postWrite);
}

public Row getConfigurationRow() {
try {
// To stay consistent with our SchemaTransform configuration naming conventions,
// we sort lexicographically
return SchemaRegistry.createDefault()
.getToRowFunction(JdbcWriteSchemaTransformConfiguration.class)
.apply(config)
.sorted()
.toSnakeCase();
} catch (NoSuchSchemaException e) {
throw new RuntimeException(e);
}
}
}

@Override
Expand Down Expand Up @@ -382,6 +398,8 @@ public static Builder builder() {
.Builder();
}

public abstract Builder toBuilder();

@AutoValue.Builder
public abstract static class Builder {
public abstract Builder setDriverClassName(String value);
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,93 @@
/*
* 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.io.jdbc.providers;

import static org.apache.beam.sdk.io.jdbc.providers.ReadFromPostgresSchemaTransformProvider.PostgresReadSchemaTransform;
import static org.apache.beam.sdk.io.jdbc.providers.WriteToPostgresSchemaTransformProvider.PostgresWriteSchemaTransform;
import static org.apache.beam.sdk.schemas.transforms.SchemaTransformTranslation.SchemaTransformPayloadTranslator;

import com.google.auto.service.AutoService;
import java.util.Map;
import org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider;
import org.apache.beam.sdk.transforms.PTransform;
import org.apache.beam.sdk.util.construction.PTransformTranslation;
import org.apache.beam.sdk.util.construction.TransformPayloadTranslatorRegistrar;
import org.apache.beam.sdk.values.Row;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap;

public class PostgresSchemaTransformTranslation {
static class PostgresReadSchemaTransformTranslator
extends SchemaTransformPayloadTranslator<PostgresReadSchemaTransform> {
@Override
public SchemaTransformProvider provider() {
return new ReadFromPostgresSchemaTransformProvider();
}

@Override
public Row toConfigRow(PostgresReadSchemaTransform transform) {
return transform.getConfigurationRow();
}
}

@AutoService(TransformPayloadTranslatorRegistrar.class)
public static class ReadRegistrar implements TransformPayloadTranslatorRegistrar {
@Override
@SuppressWarnings({
"rawtypes",
})
public Map<
? extends Class<? extends PTransform>,
? extends PTransformTranslation.TransformPayloadTranslator>
getTransformPayloadTranslators() {
return ImmutableMap
.<Class<? extends PTransform>, PTransformTranslation.TransformPayloadTranslator>builder()
.put(PostgresReadSchemaTransform.class, new PostgresReadSchemaTransformTranslator())
.build();
}
}

static class PostgresWriteSchemaTransformTranslator
extends SchemaTransformPayloadTranslator<PostgresWriteSchemaTransform> {
@Override
public SchemaTransformProvider provider() {
return new WriteToPostgresSchemaTransformProvider();
}

@Override
public Row toConfigRow(PostgresWriteSchemaTransform transform) {
return transform.getConfigurationRow();
}
}

@AutoService(TransformPayloadTranslatorRegistrar.class)
public static class WriteRegistrar implements TransformPayloadTranslatorRegistrar {
@Override
@SuppressWarnings({
"rawtypes",
})
public Map<
? extends Class<? extends PTransform>,
? extends PTransformTranslation.TransformPayloadTranslator>
getTransformPayloadTranslators() {
return ImmutableMap
.<Class<? extends PTransform>, PTransformTranslation.TransformPayloadTranslator>builder()
.put(PostgresWriteSchemaTransform.class, new PostgresWriteSchemaTransformTranslator())
.build();
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -18,20 +18,30 @@
package org.apache.beam.sdk.io.jdbc.providers;

import static org.apache.beam.sdk.io.jdbc.JdbcUtil.POSTGRES;
import static org.apache.beam.sdk.util.construction.BeamUrns.getUrn;

import com.google.auto.service.AutoService;
import java.util.Collections;
import java.util.List;
import org.apache.beam.model.pipeline.v1.ExternalTransforms;
import org.apache.beam.sdk.io.jdbc.JdbcReadSchemaTransformProvider;
import org.apache.beam.sdk.schemas.transforms.SchemaTransform;
import org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider;
import org.checkerframework.checker.initialization.qual.Initialized;
import org.checkerframework.checker.nullness.qual.NonNull;
import org.checkerframework.checker.nullness.qual.UnknownKeyFor;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

@AutoService(SchemaTransformProvider.class)
public class ReadFromPostgresSchemaTransformProvider extends JdbcReadSchemaTransformProvider {

private static final Logger LOG =
LoggerFactory.getLogger(ReadFromPostgresSchemaTransformProvider.class);

@Override
public @UnknownKeyFor @NonNull @Initialized String identifier() {
return "beam:schematransform:org.apache.beam:postgres_read:v1";
return getUrn(ExternalTransforms.ManagedTransforms.Urns.POSTGRES_READ);
}

@Override
Expand All @@ -43,4 +53,40 @@ public String description() {
protected String jdbcType() {
return POSTGRES;
}

@Override
public @UnknownKeyFor @NonNull @Initialized SchemaTransform from(
JdbcReadSchemaTransformConfiguration configuration) {
String jdbcType = configuration.getJdbcType();
if (jdbcType != null && !jdbcType.equals(jdbcType())) {
throw new IllegalArgumentException(
String.format("Wrong JDBC type. Expected '%s' but got '%s'", jdbcType(), jdbcType));
}

List<@org.checkerframework.checker.nullness.qual.Nullable String> connectionInitSql =
configuration.getConnectionInitSql();
if (connectionInitSql != null && !connectionInitSql.isEmpty()) {
LOG.warn("Postgres does not support connectionInitSql, ignoring.");
}

Boolean disableAutoCommit = configuration.getDisableAutoCommit();
if (disableAutoCommit != null && !disableAutoCommit) {
LOG.warn("Postgres reads require disableAutoCommit to be true, overriding to true.");
}

// Override "connectionInitSql" and "disableAutoCommit" for postgres
configuration =
configuration
.toBuilder()
.setConnectionInitSql(Collections.emptyList())
.setDisableAutoCommit(true)
.build();
return new PostgresReadSchemaTransform(configuration);
}

public static class PostgresReadSchemaTransform extends JdbcReadSchemaTransform {
public PostgresReadSchemaTransform(JdbcReadSchemaTransformConfiguration config) {
super(config, POSTGRES);
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -18,20 +18,30 @@
package org.apache.beam.sdk.io.jdbc.providers;

import static org.apache.beam.sdk.io.jdbc.JdbcUtil.POSTGRES;
import static org.apache.beam.sdk.util.construction.BeamUrns.getUrn;

import com.google.auto.service.AutoService;
import java.util.Collections;
import java.util.List;
import org.apache.beam.model.pipeline.v1.ExternalTransforms;
import org.apache.beam.sdk.io.jdbc.JdbcWriteSchemaTransformProvider;
import org.apache.beam.sdk.schemas.transforms.SchemaTransform;
import org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider;
import org.checkerframework.checker.initialization.qual.Initialized;
import org.checkerframework.checker.nullness.qual.NonNull;
import org.checkerframework.checker.nullness.qual.UnknownKeyFor;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

@AutoService(SchemaTransformProvider.class)
public class WriteToPostgresSchemaTransformProvider extends JdbcWriteSchemaTransformProvider {

private static final Logger LOG =
LoggerFactory.getLogger(WriteToPostgresSchemaTransformProvider.class);

@Override
public @UnknownKeyFor @NonNull @Initialized String identifier() {
return "beam:schematransform:org.apache.beam:postgres_write:v1";
return getUrn(ExternalTransforms.ManagedTransforms.Urns.POSTGRES_WRITE);
}

@Override
Expand All @@ -43,4 +53,30 @@ public String description() {
protected String jdbcType() {
return POSTGRES;
}

@Override
public @UnknownKeyFor @NonNull @Initialized SchemaTransform from(
JdbcWriteSchemaTransformConfiguration configuration) {
String jdbcType = configuration.getJdbcType();
if (jdbcType != null && !jdbcType.equals(jdbcType())) {
throw new IllegalArgumentException(
String.format("Wrong JDBC type. Expected '%s' but got '%s'", jdbcType(), jdbcType));
}

List<@org.checkerframework.checker.nullness.qual.Nullable String> connectionInitSql =
configuration.getConnectionInitSql();
if (connectionInitSql != null && !connectionInitSql.isEmpty()) {
LOG.warn("Postgres does not support connectionInitSql, ignoring.");
}

// Override "connectionInitSql" for postgres
configuration = configuration.toBuilder().setConnectionInitSql(Collections.emptyList()).build();
return new PostgresWriteSchemaTransform(configuration);
}

public static class PostgresWriteSchemaTransform extends JdbcWriteSchemaTransform {
public PostgresWriteSchemaTransform(JdbcWriteSchemaTransformConfiguration config) {
super(config, POSTGRES);
}
}
}
Loading
Loading