Skip to content
Closed
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,18 @@ 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"];
MYSQL_READ = 9 [(org.apache.beam.model.pipeline.v1.beam_urn) =
"beam:schematransform:org.apache.beam:mysql_read:v1"];
MYSQL_WRITE = 10 [(org.apache.beam.model.pipeline.v1.beam_urn) =
"beam:schematransform:org.apache.beam:mysql_write:v1"];
ORACLE_READ = 11 [(org.apache.beam.model.pipeline.v1.beam_urn) =
"beam:schematransform:org.apache.beam:oracle_read:v1"];
ORACLE_WRITE = 12 [(org.apache.beam.model.pipeline.v1.beam_urn) =
"beam:schematransform:org.apache.beam:oracle_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.ReadFromMySqlSchemaTransformProvider.MySqlReadSchemaTransform;
import static org.apache.beam.sdk.io.jdbc.providers.WriteToMySqlSchemaTransformProvider.MySqlWriteSchemaTransform;
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 MySqlSchemaTransformTranslation {
static class MySqlReadSchemaTransformTranslator
extends SchemaTransformPayloadTranslator<MySqlReadSchemaTransform> {
@Override
public SchemaTransformProvider provider() {
return new ReadFromMySqlSchemaTransformProvider();
}

@Override
public Row toConfigRow(MySqlReadSchemaTransform 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(MySqlReadSchemaTransform.class, new MySqlReadSchemaTransformTranslator())
.build();
}
}

static class MySqlWriteSchemaTransformTranslator
extends SchemaTransformPayloadTranslator<MySqlWriteSchemaTransform> {
@Override
public SchemaTransformProvider provider() {
return new WriteToMySqlSchemaTransformProvider();
}

@Override
public Row toConfigRow(MySqlWriteSchemaTransform 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(MySqlWriteSchemaTransform.class, new MySqlWriteSchemaTransformTranslator())
.build();
}
}
}
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.ReadFromOracleSchemaTransformProvider.OracleReadSchemaTransform;
import static org.apache.beam.sdk.io.jdbc.providers.WriteToOracleSchemaTransformProvider.OracleWriteSchemaTransform;
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 OracleSchemaTransformTranslation {
static class OracleReadSchemaTransformTranslator
extends SchemaTransformPayloadTranslator<OracleReadSchemaTransform> {
@Override
public SchemaTransformProvider provider() {
return new ReadFromOracleSchemaTransformProvider();
}

@Override
public Row toConfigRow(OracleReadSchemaTransform 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(OracleReadSchemaTransform.class, new OracleReadSchemaTransformTranslator())
.build();
}
}

static class OracleWriteSchemaTransformTranslator
extends SchemaTransformPayloadTranslator<OracleWriteSchemaTransform> {
@Override
public SchemaTransformProvider provider() {
return new WriteToOracleSchemaTransformProvider();
}

@Override
public Row toConfigRow(OracleWriteSchemaTransform 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(OracleWriteSchemaTransform.class, new OracleWriteSchemaTransformTranslator())
.build();
}
}
}
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();
}
}
}
Loading
Loading