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 @@ -84,6 +84,10 @@ message ManagedTransforms {
"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"];
SQL_SERVER_READ = 11 [(org.apache.beam.model.pipeline.v1.beam_urn) =
"beam:schematransform:org.apache.beam:sql_server_read:v1"];
SQL_SERVER_WRITE = 12 [(org.apache.beam.model.pipeline.v1.beam_urn) =
"beam:schematransform:org.apache.beam:sql_server_write:v1"];
}
}

Expand Down
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.MSSQL;
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 ReadFromSqlServerSchemaTransformProvider extends JdbcReadSchemaTransformProvider {

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

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

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

@Override
public @UnknownKeyFor @NonNull @Initialized SchemaTransform from(
JdbcReadSchemaTransformConfiguration configuration) {
String jdbcType = configuration.getJdbcType();
if (jdbcType != null && !jdbcType.isEmpty() && !jdbcType.equals(jdbcType())) {
LOG.warn(
"Wrong JDBC type. Expected '{}' but got '{}'. Overriding with '{}'.",
jdbcType(),
jdbcType,
jdbcType());
configuration = configuration.toBuilder().setJdbcType(jdbcType()).build();
}

List<@org.checkerframework.checker.nullness.qual.Nullable String> connectionInitSql =
configuration.getConnectionInitSql();
if (connectionInitSql != null && !connectionInitSql.isEmpty()) {
throw new IllegalArgumentException("SQL Server does not support connectionInitSql.");
}

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

public static class SqlServerReadSchemaTransform extends JdbcReadSchemaTransform {
public SqlServerReadSchemaTransform(JdbcReadSchemaTransformConfiguration config) {
super(config, MSSQL);
config.validate(MSSQL);
}
}
}
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.ReadFromSqlServerSchemaTransformProvider.SqlServerReadSchemaTransform;
import static org.apache.beam.sdk.io.jdbc.providers.WriteToSqlServerSchemaTransformProvider.SqlServerWriteSchemaTransform;
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 SqlServerSchemaTransformTranslation {
static class SqlServerReadSchemaTransformTranslator
extends SchemaTransformPayloadTranslator<SqlServerReadSchemaTransform> {
@Override
public SchemaTransformProvider provider() {
return new ReadFromSqlServerSchemaTransformProvider();
}

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

static class SqlServerWriteSchemaTransformTranslator
extends SchemaTransformPayloadTranslator<SqlServerWriteSchemaTransform> {
@Override
public SchemaTransformProvider provider() {
return new WriteToSqlServerSchemaTransformProvider();
}

@Override
public Row toConfigRow(SqlServerWriteSchemaTransform 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(SqlServerWriteSchemaTransform.class, new SqlServerWriteSchemaTransformTranslator())
.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.MSSQL;
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 WriteToSqlServerSchemaTransformProvider extends JdbcWriteSchemaTransformProvider {

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

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

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

@Override
public @UnknownKeyFor @NonNull @Initialized SchemaTransform from(
JdbcWriteSchemaTransformConfiguration configuration) {
String jdbcType = configuration.getJdbcType();
if (jdbcType != null && !jdbcType.isEmpty() && !jdbcType.equals(jdbcType())) {
LOG.warn(
"Wrong JDBC type. Expected '{}' but got '{}'. Overriding with '{}'.",
jdbcType(),
jdbcType,
jdbcType());
configuration = configuration.toBuilder().setJdbcType(jdbcType()).build();
}

List<@org.checkerframework.checker.nullness.qual.Nullable String> connectionInitSql =
configuration.getConnectionInitSql();
if (connectionInitSql != null && !connectionInitSql.isEmpty()) {
throw new IllegalArgumentException("SQL Server does not support connectionInitSql.");
}

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

public static class SqlServerWriteSchemaTransform extends JdbcWriteSchemaTransform {
public SqlServerWriteSchemaTransform(JdbcWriteSchemaTransformConfiguration config) {
super(config, MSSQL);
config.validate(MSSQL);
}
}
}
Loading
Loading