From e97f31935132dc2a7c1d13fff3d8af9cf1edb122 Mon Sep 17 00:00:00 2001 From: Adalbert Makarovych Date: Mon, 14 Nov 2022 16:04:45 +0200 Subject: [PATCH 01/21] Implemented SchemaTransforms for SingleStoreIO Added default RowMapper and UserDataMapper These changes will allow to configure SingleStoreIO easier and to use it with other languages --- .../job_PerformanceTests_SingleStoreIO.groovy | 2 +- ...ob_PostCommit_Java_SingleStoreIO_IT.groovy | 86 ++++++ .../SingleStoreDefaultRowMapper.java | 272 ++++++++++++++++++ .../SingleStoreDefaultUserDataMapper.java | 108 +++++++ .../sdk/io/singlestore/SingleStoreIO.java | 254 +++++++++++++++- ...StoreSchemaTransformReadConfiguration.java | 79 +++++ ...ingleStoreSchemaTransformReadProvider.java | 140 +++++++++ ...nsformReadWithPartitionsConfiguration.java | 84 ++++++ ...maTransformReadWithPartitionsProvider.java | 144 ++++++++++ ...toreSchemaTransformWriteConfiguration.java | 74 +++++ ...ngleStoreSchemaTransformWriteProvider.java | 152 ++++++++++ .../sdk/io/singlestore/SingleStoreUtil.java | 5 + .../SingleStoreDefaultRowMapperTest.java | 235 +++++++++++++++ .../SingleStoreDefaultUserDataMapperTest.java | 136 +++++++++ .../SingleStoreIODefaultMapperIT.java | 252 ++++++++++++++++ ...T.java => SingleStoreIOPerformanceIT.java} | 21 +- .../SingleStoreIOSchemaTransformIT.java | 267 +++++++++++++++++ .../io/singlestore/SingleStoreUtilTest.java | 13 +- .../beam/sdk/io/singlestore/TestHelper.java | 19 ++ 19 files changed, 2321 insertions(+), 22 deletions(-) create mode 100644 .test-infra/jenkins/job_PostCommit_Java_SingleStoreIO_IT.groovy create mode 100644 sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreDefaultRowMapper.java create mode 100644 sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreDefaultUserDataMapper.java create mode 100644 sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreSchemaTransformReadConfiguration.java create mode 100644 sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreSchemaTransformReadProvider.java create mode 100644 sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreSchemaTransformReadWithPartitionsConfiguration.java create mode 100644 sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreSchemaTransformReadWithPartitionsProvider.java create mode 100644 sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreSchemaTransformWriteConfiguration.java create mode 100644 sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreSchemaTransformWriteProvider.java create mode 100644 sdks/java/io/singlestore/src/test/java/org/apache/beam/sdk/io/singlestore/SingleStoreDefaultRowMapperTest.java create mode 100644 sdks/java/io/singlestore/src/test/java/org/apache/beam/sdk/io/singlestore/SingleStoreDefaultUserDataMapperTest.java create mode 100644 sdks/java/io/singlestore/src/test/java/org/apache/beam/sdk/io/singlestore/SingleStoreIODefaultMapperIT.java rename sdks/java/io/singlestore/src/test/java/org/apache/beam/sdk/io/singlestore/{SingleStoreIOIT.java => SingleStoreIOPerformanceIT.java} (92%) create mode 100644 sdks/java/io/singlestore/src/test/java/org/apache/beam/sdk/io/singlestore/SingleStoreIOSchemaTransformIT.java diff --git a/.test-infra/jenkins/job_PerformanceTests_SingleStoreIO.groovy b/.test-infra/jenkins/job_PerformanceTests_SingleStoreIO.groovy index a5fbc278a24e..2d6df33f4f1b 100644 --- a/.test-infra/jenkins/job_PerformanceTests_SingleStoreIO.groovy +++ b/.test-infra/jenkins/job_PerformanceTests_SingleStoreIO.groovy @@ -80,7 +80,7 @@ job(jobName) { switches("--info") switches("-DintegrationTestPipelineOptions=\'${common.joinPipelineOptions(pipelineOptions)}\'") switches("-DintegrationTestRunner=dataflow") - tasks(":sdks:java:io:singlestore:integrationTest --tests org.apache.beam.sdk.io.singlestore.SingleStoreIOIT") + tasks(":sdks:java:io:singlestore:integrationTest --tests org.apache.beam.sdk.io.singlestore.SingleStoreIOITPerformance") } } } diff --git a/.test-infra/jenkins/job_PostCommit_Java_SingleStoreIO_IT.groovy b/.test-infra/jenkins/job_PostCommit_Java_SingleStoreIO_IT.groovy new file mode 100644 index 000000000000..23664ee4ba79 --- /dev/null +++ b/.test-infra/jenkins/job_PostCommit_Java_SingleStoreIO_IT.groovy @@ -0,0 +1,86 @@ +/* + * 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. + */ + +import CommonJobProperties as common +import PostcommitJobBuilder +import Kubernetes + +String jobName = "beam_PostCommit_Java_SingleStoreIO_IT" + +void waitForPodWithLabel(job, Kubernetes k8s, String label) { + job.steps { + shell("${k8s.KUBERNETES_DIR}/singlestore/wait-for-pod-with-label.sh ${label} 600") + } +} + +void waitFor(job, Kubernetes k8s, String resource) { + job.steps { + shell("${k8s.KUBERNETES_DIR}/singlestore/wait-for.sh ${resource} 600") + } +} + + +// This job runs the integration test of java SingleStoreIO class. +PostcommitJobBuilder.postCommitJob(jobName, + 'Run Java SingleStoreIO_IT', 'Java SingleStoreIO Integration Test',this) { + description('Runs the Java SingleStoreIO Integration Test.') + + // Set common parameters. + commonJobProperties.setTopLevelMainJobProperties(delegate) + + // Deploy SingleStoreDB cluster + String namespace = common.getKubernetesNamespace(jobName) + String kubeconfigPath = common.getKubeconfigLocationForNamespace(namespace) + Kubernetes k8s = Kubernetes.create(delegate, kubeconfigPath, namespace) + + k8s.apply(common.makePathAbsolute("src/.test-infra/kubernetes/singlestore/sdb-rbac.yaml")) + k8s.apply(common.makePathAbsolute("src/.test-infra/kubernetes/singlestore/sdb-cluster-crd.yaml")) + k8s.apply(common.makePathAbsolute("src/.test-infra/kubernetes/singlestore/sdb-operator.yaml")) + waitForPodWithLabel(delegate, k8s, "sdb-operator") + + k8s.apply(common.makePathAbsolute("src/.test-infra/kubernetes/singlestore/sdb-cluster.yaml")) + waitFor(delegate, k8s, "memsqlclusters.memsql.com") + + String singlestoreHostName = "LOAD_BALANCER_IP" + k8s.loadBalancerIP("svc-sdb-cluster-ddl", singlestoreHostName) + + // Define test options + Map pipelineOptions = [ + tempRoot : 'gs://temp-storage-for-perf-tests', + project : 'apache-beam-testing', + runner : 'DataflowRunner', + singleStoreServerName : singlestoreHostName, + singleStoreUsername : "admin", + singleStorePassword : "secretpass", + singleStorePort: "3306", + numberOfRecords: "1000", + ] + + // Gradle goals for this job. + steps { + gradle { + rootBuildScriptDir(common.checkoutDir) + common.setGradleSwitches(delegate) + switches("--info") + switches("-DintegrationTestPipelineOptions=\'${common.joinPipelineOptions(pipelineOptions)}\'") + switches("-DintegrationTestRunner=dataflow") + tasks(":sdks:java:io:singlestore:integrationTest --tests org.apache.beam.sdk.io.singlestore.SingleStoreIODefaultMapperIT") + tasks(":sdks:java:io:singlestore:integrationTest --tests org.apache.beam.sdk.io.singlestore.SingleStoreIOSchemaTransformIT") + } + } + } diff --git a/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreDefaultRowMapper.java b/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreDefaultRowMapper.java new file mode 100644 index 000000000000..e10da5acaf40 --- /dev/null +++ b/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreDefaultRowMapper.java @@ -0,0 +1,272 @@ +/* + * 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.singlestore; + +import static java.sql.Types.BIGINT; +import static java.sql.Types.BINARY; +import static java.sql.Types.BIT; +import static java.sql.Types.CHAR; +import static java.sql.Types.DATE; +import static java.sql.Types.DECIMAL; +import static java.sql.Types.DOUBLE; +import static java.sql.Types.INTEGER; +import static java.sql.Types.LONGVARBINARY; +import static java.sql.Types.LONGVARCHAR; +import static java.sql.Types.NULL; +import static java.sql.Types.REAL; +import static java.sql.Types.SMALLINT; +import static java.sql.Types.TIME; +import static java.sql.Types.TIMESTAMP; +import static java.sql.Types.TINYINT; +import static java.sql.Types.VARBINARY; +import static java.sql.Types.VARCHAR; +import static org.apache.beam.sdk.schemas.Schema.FieldType.BOOLEAN; +import static org.apache.beam.sdk.schemas.Schema.FieldType.BYTE; +import static org.apache.beam.sdk.schemas.Schema.FieldType.FLOAT; +import static org.apache.beam.sdk.schemas.Schema.FieldType.INT16; +import static org.apache.beam.sdk.schemas.Schema.FieldType.INT32; +import static org.apache.beam.sdk.schemas.Schema.FieldType.INT64; +import static org.apache.beam.sdk.schemas.Schema.FieldType.STRING; + +import java.io.Serializable; +import java.sql.ResultSet; +import java.sql.ResultSetMetaData; +import java.sql.SQLException; +import java.sql.Time; +import java.sql.Timestamp; +import java.time.ZoneOffset; +import java.time.ZonedDateTime; +import java.util.ArrayList; +import java.util.Calendar; +import java.util.List; +import java.util.TimeZone; +import org.apache.beam.sdk.schemas.Schema; +import org.apache.beam.sdk.schemas.logicaltypes.VariableBytes; +import org.apache.beam.sdk.schemas.logicaltypes.VariableString; +import org.apache.beam.sdk.values.Row; +import org.joda.time.DateTime; +import org.joda.time.LocalDate; +import org.joda.time.chrono.ISOChronology; + +class SingleStoreDefaultRowMapper implements SingleStoreIO.RowMapper { + Schema schema; + + List converters; + + public SingleStoreDefaultRowMapper(ResultSetMetaData metaData) throws SQLException { + converters = new ArrayList<>(); + for (int i = 0; i < metaData.getColumnCount(); i++) { + converters.add(ResultSetFieldConverter.of(metaData.getColumnType(i + 1))); + } + + Schema.Builder schemaBuilder = new Schema.Builder(); + for (int i = 0; i < metaData.getColumnCount(); i++) { + schemaBuilder.addField(converters.get(i).getSchemaField(metaData, i + 1)); + } + this.schema = schemaBuilder.build(); + } + + public Schema getSchema() { + return this.schema; + } + + @Override + public Row mapRow(ResultSet resultSet) throws Exception { + Row.Builder rowBuilder = Row.withSchema(schema); + + for (int i = 0; i < schema.getFieldCount(); i++) { + Object value = converters.get(i).getValue(resultSet, i + 1); + + if (resultSet.wasNull()) { + rowBuilder.addValue(null); + } else { + rowBuilder.addValue(value); + } + } + + return rowBuilder.build(); + } + + abstract static class ResultSetFieldConverter implements Serializable { + abstract Object getValue(ResultSet rs, Integer index) throws SQLException; + + Schema.Field getSchemaField(ResultSetMetaData md, Integer index) throws SQLException { + String label = md.getColumnLabel(index); + return Schema.Field.of(label, getSchemaFieldType(md, index)) + .withNullable(md.isNullable(index) == java.sql.ResultSetMetaData.columnNullable); + } + + abstract Schema.FieldType getSchemaFieldType(ResultSetMetaData md, Integer index) + throws SQLException; + + /** + * Interface implemented by functions that extract values of different types from a JDBC + * ResultSet. + */ + @FunctionalInterface + interface ResultSetFieldExtractor extends Serializable { + Object extract(ResultSet rs, Integer index) throws SQLException; + } + + static ResultSetFieldConverter of(int columnType) { + switch (columnType) { + case BIT: + return new DirectResultSetFieldConverter(BOOLEAN, ResultSet::getBoolean); + case TINYINT: + return new DirectResultSetFieldConverter(BYTE, ResultSet::getByte); + case SMALLINT: + return new DirectResultSetFieldConverter(INT16, ResultSet::getShort); + case INTEGER: + return new DirectResultSetFieldConverter(INT32, ResultSet::getInt); + case BIGINT: + return new DirectResultSetFieldConverter(INT64, ResultSet::getLong); + case REAL: + return new DirectResultSetFieldConverter(FLOAT, ResultSet::getFloat); + case DOUBLE: + return new DirectResultSetFieldConverter(Schema.FieldType.DOUBLE, ResultSet::getDouble); + case DECIMAL: + return new DirectResultSetFieldConverter( + Schema.FieldType.DECIMAL, ResultSet::getBigDecimal); + case TIMESTAMP: + return new TimestampResultSetFieldConverter(); + case DATE: + return new DateResultSetFieldConverter(); + case TIME: + return new TimeResultSetFieldConverter(); + case LONGVARBINARY: + case VARBINARY: + case BINARY: + return new BinaryResultSetFieldConverter(); + case LONGVARCHAR: + case VARCHAR: + case CHAR: + return new CharResultSetFieldConverter(); + case NULL: + return new DirectResultSetFieldConverter(STRING, ResultSet::getString); + default: + throw new UnsupportedOperationException( + "Converting " + columnType + " to Beam schema type is not supported"); + } + } + } + + static class DirectResultSetFieldConverter extends ResultSetFieldConverter { + Schema.FieldType fieldType; + ResultSetFieldExtractor extractor; + + public DirectResultSetFieldConverter( + Schema.FieldType fieldType, ResultSetFieldExtractor extractor) { + this.fieldType = fieldType; + this.extractor = extractor; + } + + @Override + Object getValue(ResultSet rs, Integer index) throws SQLException { + return extractor.extract(rs, index); + } + + @Override + Schema.FieldType getSchemaFieldType(ResultSetMetaData md, Integer index) { + return fieldType; + } + } + + static class CharResultSetFieldConverter extends ResultSetFieldConverter { + @Override + Object getValue(ResultSet rs, Integer index) throws SQLException { + return rs.getString(index); + } + + @Override + Schema.FieldType getSchemaFieldType(ResultSetMetaData md, Integer index) throws SQLException { + int size = md.getPrecision(index); + return Schema.FieldType.logicalType(VariableString.of(size)); + } + } + + static class BinaryResultSetFieldConverter extends ResultSetFieldConverter { + @Override + Object getValue(ResultSet rs, Integer index) throws SQLException { + return rs.getBytes(index); + } + + @Override + Schema.FieldType getSchemaFieldType(ResultSetMetaData md, Integer index) throws SQLException { + int size = md.getPrecision(index); + return Schema.FieldType.logicalType(VariableBytes.of(size)); + } + } + + static class TimestampResultSetFieldConverter extends ResultSetFieldConverter { + @Override + Object getValue(ResultSet rs, Integer index) throws SQLException { + Timestamp ts = + rs.getTimestamp(index, Calendar.getInstance(TimeZone.getTimeZone(ZoneOffset.UTC))); + if (ts == null) { + return null; + } + return new DateTime(ts.toInstant().toEpochMilli(), ISOChronology.getInstanceUTC()); + } + + @Override + Schema.FieldType getSchemaFieldType(ResultSetMetaData md, Integer index) { + return Schema.FieldType.DATETIME; + } + } + + static class TimeResultSetFieldConverter extends ResultSetFieldConverter { + @Override + Object getValue(ResultSet rs, Integer index) throws SQLException { + Time time = rs.getTime(index, Calendar.getInstance(TimeZone.getTimeZone(ZoneOffset.UTC))); + if (time == null) { + return null; + } + return new DateTime(time.getTime(), ISOChronology.getInstanceUTC()) + .withDate(new LocalDate(0L)); + } + + @Override + Schema.FieldType getSchemaFieldType(ResultSetMetaData md, Integer index) { + return Schema.FieldType.DATETIME; + } + } + + static class DateResultSetFieldConverter extends ResultSetFieldConverter { + @Override + Object getValue(ResultSet rs, Integer index) throws SQLException { + // TODO(https://github.com/apache/beam/issues/19215) import when joda LocalDate is removed. + java.time.LocalDate date = rs.getObject(index, java.time.LocalDate.class); + if (date == null) { + return null; + } + ZonedDateTime zdt = date.atStartOfDay(ZoneOffset.UTC); + return new DateTime(zdt.toInstant().toEpochMilli(), ISOChronology.getInstanceUTC()); + } + + @Override + Schema.FieldType getSchemaFieldType(ResultSetMetaData md, Integer index) { + return Schema.FieldType.DATETIME; + } + } + + public static class SingleStoreDefaultRowMapperCreationException extends RuntimeException { + SingleStoreDefaultRowMapperCreationException(String message, Throwable cause) { + super(message, cause); + } + } +} diff --git a/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreDefaultUserDataMapper.java b/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreDefaultUserDataMapper.java new file mode 100644 index 000000000000..e7d4760d2bbc --- /dev/null +++ b/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreDefaultUserDataMapper.java @@ -0,0 +1,108 @@ +/* + * 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.singlestore; + +import java.math.BigDecimal; +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.List; +import org.apache.beam.sdk.schemas.Schema; +import org.apache.beam.sdk.values.Row; +import org.joda.time.Instant; +import org.joda.time.format.DateTimeFormat; +import org.joda.time.format.DateTimeFormatter; + +final class SingleStoreDefaultUserDataMapper implements SingleStoreIO.UserDataMapper { + + private DateTimeFormatter formatter = null; + + private DateTimeFormatter getFormatter() { + if (formatter == null) { + formatter = DateTimeFormat.forPattern("yyyy-MM-DD' 'HH:mm:ss.SSS"); + } + + return formatter; + } + + private String convertLogicalTypeFieldToString(Schema.FieldType type, Object value) { + Schema.LogicalType logicalType = type.getLogicalType(); + assert logicalType != null; + + Schema.FieldType baseType = logicalType.getBaseType(); + Object baseValue = logicalType.toBaseType(value); + return convertFieldToString(baseType, baseValue); + } + + private String convertFieldToString(Schema.FieldType type, Object value) { + switch (type.getTypeName()) { + case BYTE: + return ((Byte) value).toString(); + case INT16: + return ((Short) value).toString(); + case INT32: + return ((Integer) value).toString(); + case INT64: + return ((Long) value).toString(); + case DECIMAL: + return ((BigDecimal) value).toString(); + case FLOAT: + return ((Float) value).toString(); + case DOUBLE: + return ((Double) value).toString(); + case STRING: + return (String) value; + case DATETIME: + return getFormatter().print((Instant) value); + case BOOLEAN: + return ((Boolean) value) ? "1" : "0"; + case BYTES: + return new String((byte[]) value, StandardCharsets.UTF_8); + case ARRAY: + throw new UnsupportedOperationException( + "Writing of ARRAY type is not supported by the default UserDataMapper"); + case ITERABLE: + throw new UnsupportedOperationException( + "Writing of ITERABLE type is not supported by the default UserDataMapper"); + case MAP: + throw new UnsupportedOperationException( + "Writing of MAP type is not supported by the default UserDataMapper"); + case ROW: + throw new UnsupportedOperationException( + "Writing of nested ROW type is not supported by the default UserDataMapper"); + case LOGICAL_TYPE: + return convertLogicalTypeFieldToString(type, value); + default: + throw new UnsupportedOperationException( + String.format( + "Writing of %s type is not supported by the default UserDataMapper", + type.getTypeName().name())); + } + } + + @Override + public List mapRow(Row element) { + List res = new ArrayList<>(); + + Schema s = element.getSchema(); + for (int i = 0; i < s.getFieldCount(); i++) { + res.add(convertFieldToString(s.getField(i).getType(), element.getValue(i))); + } + + return res; + } +} diff --git a/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreIO.java b/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreIO.java index 6873ae6b8b37..22668901f56d 100644 --- a/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreIO.java +++ b/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreIO.java @@ -30,12 +30,15 @@ import java.sql.Connection; import java.sql.PreparedStatement; import java.sql.ResultSet; +import java.sql.ResultSetMetaData; +import java.sql.SQLException; import java.sql.Statement; import java.util.ArrayList; import java.util.List; import javax.sql.DataSource; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.ListCoder; +import org.apache.beam.sdk.coders.RowCoder; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.io.range.OffsetRange; import org.apache.beam.sdk.transforms.Create; @@ -53,6 +56,7 @@ import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; +import org.apache.beam.sdk.values.Row; import org.apache.commons.dbcp2.BasicDataSource; import org.apache.commons.dbcp2.DelegatingStatement; import org.checkerframework.checker.nullness.qual.Nullable; @@ -168,6 +172,11 @@ public static Read read() { return new AutoValue_SingleStoreIO_Read.Builder().setOutputParallelization(true).build(); } + /** Read Beam {@link Row}s from a SingleStoreDB datasource. */ + public static ReadRows readRows() { + return new AutoValue_SingleStoreIO_ReadRows.Builder().setOutputParallelization(true).build(); + } + /** * Like {@link #read}, but executes multiple instances of the query on the same table for each * database partition. @@ -178,6 +187,14 @@ public static ReadWithPartitions readWithPartitions() { return new AutoValue_SingleStoreIO_ReadWithPartitions.Builder().build(); } + /** + * Like {@link #readRows}, but executes multiple instances of the query on the same table for each + * database partition. + */ + public static ReadWithPartitionsRows readWithPartitionsRows() { + return new AutoValue_SingleStoreIO_ReadWithPartitionsRows.Builder().build(); + } + /** * Write data to a SingleStoreDB datasource. * @@ -187,6 +204,13 @@ public static Write write() { return new AutoValue_SingleStoreIO_Write.Builder().build(); } + /** Write Beam {@link Row}s to a SingleStoreDB datasource. */ + public static Write writeRows() { + return new AutoValue_SingleStoreIO_Write.Builder() + .setUserDataMapper(new SingleStoreDefaultUserDataMapper()) + .build(); + } + /** * An interface used by {@link Read} and {@link ReadWithPartitions} for converting each row of the * {@link ResultSet} into an element of the resulting {@link PCollection}. @@ -303,7 +327,8 @@ public DataSource getDataSource() { String database = SingleStoreUtil.getArgumentWithDefault(getDatabase(), ""); String connectionProperties = SingleStoreUtil.getArgumentWithDefault(getConnectionProperties(), ""); - connectionProperties += (connectionProperties.isEmpty() ? "" : ";") + "allowLocalInfile=TRUE"; + connectionProperties += + (connectionProperties.isEmpty() ? "" : ";") + "allowLocalInfile=TRUE"; String username = getUsername(); String password = getPassword(); @@ -343,6 +368,8 @@ public abstract static class Read extends PTransform> abstract @Nullable RowMapper getRowMapper(); + abstract @Nullable Coder getCoder(); + abstract Builder toBuilder(); @AutoValue.Builder @@ -360,6 +387,8 @@ abstract Builder setDataSourceConfiguration( abstract Builder setRowMapper(RowMapper rowMapper); + abstract Builder setCoder(Coder coder); + abstract Read build(); } @@ -397,6 +426,11 @@ public Read withRowMapper(RowMapper rowMapper) { return toBuilder().setRowMapper(rowMapper).build(); } + public Read withCoder(Coder coder) { + checkNotNull(coder, "coder can not be null"); + return toBuilder().setCoder(coder).build(); + } + @Override public PCollection expand(PBegin input) { DataSourceConfiguration dataSourceConfiguration = getDataSourceConfiguration(); @@ -408,6 +442,7 @@ public PCollection expand(PBegin input) { Coder coder = SingleStoreUtil.inferCoder( + getCoder(), rowMapper, input.getPipeline().getCoderRegistry(), input.getPipeline().getSchemaRegistry(), @@ -528,6 +563,130 @@ public void populateDisplayData(DisplayData.Builder builder) { } } + private static SingleStoreDefaultRowMapper getRowMapper( + DataSourceConfiguration dataSourceConfiguration, String query) { + try { + DataSource dataSource = dataSourceConfiguration.getDataSource(); + Connection conn = dataSource.getConnection(); + try { + PreparedStatement stmt = + conn.prepareStatement(String.format("SELECT * FROM (%s) LIMIT 0", query)); + try { + ResultSetMetaData md = stmt.getMetaData(); + return new SingleStoreDefaultRowMapper(md); + } finally { + stmt.close(); + } + } finally { + conn.close(); + } + } catch (SQLException e) { + throw new SingleStoreDefaultRowMapper.SingleStoreDefaultRowMapperCreationException( + "Failed to create default row mapper", e); + } + } + + @AutoValue + public abstract static class ReadRows extends PTransform> { + abstract @Nullable DataSourceConfiguration getDataSourceConfiguration(); + + abstract @Nullable String getQuery(); + + abstract @Nullable String getTable(); + + abstract @Nullable StatementPreparator getStatementPreparator(); + + abstract @Nullable Boolean getOutputParallelization(); + + abstract Builder toBuilder(); + + @AutoValue.Builder + abstract static class Builder { + abstract Builder setDataSourceConfiguration(DataSourceConfiguration dataSourceConfiguration); + + abstract Builder setQuery(String query); + + abstract Builder setTable(String table); + + abstract Builder setStatementPreparator(StatementPreparator statementPreparator); + + abstract Builder setOutputParallelization(Boolean outputParallelization); + + abstract ReadRows build(); + } + + public ReadRows withDataSourceConfiguration(DataSourceConfiguration config) { + checkNotNull(config, "dataSourceConfiguration can not be null"); + return toBuilder().setDataSourceConfiguration(config).build(); + } + + public ReadRows withQuery(String query) { + checkNotNull(query, "query can not be null"); + return toBuilder().setQuery(query).build(); + } + + public ReadRows withTable(String table) { + checkNotNull(table, "table can not be null"); + return toBuilder().setTable(table).build(); + } + + public ReadRows withStatementPreparator(StatementPreparator statementPreparator) { + checkNotNull(statementPreparator, "statementPreparator can not be null"); + return toBuilder().setStatementPreparator(statementPreparator).build(); + } + + /** + * Whether to reshuffle the resulting PCollection so results are distributed to all workers. The + * default is to parallelize and should only be changed if this is known to be unnecessary. + */ + public ReadRows withOutputParallelization(Boolean outputParallelization) { + checkNotNull(outputParallelization, "outputParallelization can not be null"); + return toBuilder().setOutputParallelization(outputParallelization).build(); + } + + @Override + public PCollection expand(PBegin input) { + DataSourceConfiguration dataSourceConfiguration = getDataSourceConfiguration(); + Preconditions.checkArgumentNotNull( + dataSourceConfiguration, "withDataSourceConfiguration() is required"); + String actualQuery = SingleStoreUtil.getSelectQuery(getTable(), getQuery()); + + SingleStoreDefaultRowMapper rowMapper = getRowMapper(dataSourceConfiguration, actualQuery); + + Read read = + SingleStoreIO.read() + .withDataSourceConfiguration(dataSourceConfiguration) + .withQuery(actualQuery) + .withRowMapper(rowMapper) + .withCoder(RowCoder.of(rowMapper.getSchema())); + + if (getOutputParallelization() != null) { + read = read.withOutputParallelization(getOutputParallelization()); + } + + if (getStatementPreparator() != null) { + read = read.withStatementPreparator(getStatementPreparator()); + } + + PCollection output = input.apply(read); + + return output.setRowSchema(rowMapper.getSchema()); + } + + @Override + public void populateDisplayData(DisplayData.Builder builder) { + super.populateDisplayData(builder); + + DataSourceConfiguration.populateDisplayData(getDataSourceConfiguration(), builder); + builder.addIfNotNull(DisplayData.item("query", getQuery())); + builder.addIfNotNull(DisplayData.item("table", getTable())); + builder.addIfNotNull( + DisplayData.item( + "statementPreparator", SingleStoreUtil.getClassNameOrNull(getStatementPreparator()))); + builder.addIfNotNull(DisplayData.item("outputParallelization", getOutputParallelization())); + } + } + /** * A {@link PTransform} for reading data from SingleStoreDB. It is used by {@link * SingleStoreIO#readWithPartitions()}. {@link ReadWithPartitions} is preferred over {@link Read} @@ -547,6 +706,8 @@ public abstract static class ReadWithPartitions extends PTransform toBuilder(); + abstract @Nullable Coder getCoder(); + @AutoValue.Builder abstract static class Builder { abstract Builder setDataSourceConfiguration( @@ -558,6 +719,8 @@ abstract Builder setDataSourceConfiguration( abstract Builder setRowMapper(RowMapper rowMapper); + abstract Builder setCoder(Coder coder); + abstract ReadWithPartitions build(); } @@ -581,6 +744,11 @@ public ReadWithPartitions withRowMapper(RowMapper rowMapper) { return toBuilder().setRowMapper(rowMapper).build(); } + public ReadWithPartitions withCoder(Coder coder) { + checkNotNull(coder, "coder can not be null"); + return toBuilder().setCoder(coder).build(); + } + @Override public PCollection expand(PBegin input) { DataSourceConfiguration dataSourceConfiguration = getDataSourceConfiguration(); @@ -597,6 +765,7 @@ public PCollection expand(PBegin input) { Coder coder = SingleStoreUtil.inferCoder( + getCoder(), rowMapper, input.getPipeline().getCoderRegistry(), input.getPipeline().getSchemaRegistry(), @@ -715,6 +884,89 @@ public void populateDisplayData(DisplayData.Builder builder) { } } + @AutoValue + public abstract static class ReadWithPartitionsRows extends PTransform> { + abstract @Nullable DataSourceConfiguration getDataSourceConfiguration(); + + abstract @Nullable String getQuery(); + + abstract @Nullable String getTable(); + + abstract @Nullable Integer getInitialNumReaders(); + + abstract ReadWithPartitionsRows.Builder toBuilder(); + + @AutoValue.Builder + abstract static class Builder { + abstract ReadWithPartitionsRows.Builder setDataSourceConfiguration( + DataSourceConfiguration dataSourceConfiguration); + + abstract Builder setQuery(String query); + + abstract Builder setTable(String table); + + abstract Builder setInitialNumReaders(Integer initialNumReaders); + + abstract ReadWithPartitionsRows build(); + } + + public ReadWithPartitionsRows withDataSourceConfiguration(DataSourceConfiguration config) { + checkNotNull(config, "dataSourceConfiguration can not be null"); + return toBuilder().setDataSourceConfiguration(config).build(); + } + + public ReadWithPartitionsRows withQuery(String query) { + checkNotNull(query, "query can not be null"); + return toBuilder().setQuery(query).build(); + } + + public ReadWithPartitionsRows withTable(String table) { + checkNotNull(table, "table can not be null"); + return toBuilder().setTable(table).build(); + } + + /** Pre-split initial restriction and start initialNumReaders reading at the very beginning. */ + public ReadWithPartitionsRows withInitialNumReaders(Integer initialNumReaders) { + checkNotNull(initialNumReaders, "initialNumReaders can not be null"); + return toBuilder().setInitialNumReaders(initialNumReaders).build(); + } + + @Override + public PCollection expand(PBegin input) { + DataSourceConfiguration dataSourceConfiguration = getDataSourceConfiguration(); + Preconditions.checkArgumentNotNull( + dataSourceConfiguration, "withDataSourceConfiguration() is required"); + String actualQuery = SingleStoreUtil.getSelectQuery(getTable(), getQuery()); + + SingleStoreDefaultRowMapper rowMapper = getRowMapper(dataSourceConfiguration, actualQuery); + + ReadWithPartitions readWithPartitions = + SingleStoreIO.readWithPartitions() + .withDataSourceConfiguration(dataSourceConfiguration) + .withQuery(actualQuery) + .withRowMapper(rowMapper) + .withCoder(RowCoder.of(rowMapper.getSchema())); + + if (getInitialNumReaders() != null) { + readWithPartitions = readWithPartitions.withInitialNumReaders(getInitialNumReaders()); + } + + PCollection output = input.apply(readWithPartitions); + + return output.setRowSchema(rowMapper.getSchema()); + } + + @Override + public void populateDisplayData(DisplayData.Builder builder) { + super.populateDisplayData(builder); + + DataSourceConfiguration.populateDisplayData(getDataSourceConfiguration(), builder); + builder.addIfNotNull(DisplayData.item("query", getQuery())); + builder.addIfNotNull(DisplayData.item("table", getTable())); + builder.addIfNotNull(DisplayData.item("initialNumReaders", getInitialNumReaders())); + } + } + /** * A {@link PTransform} for writing data to SingleStoreDB. It is used by {@link * SingleStoreIO#write()}. diff --git a/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreSchemaTransformReadConfiguration.java b/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreSchemaTransformReadConfiguration.java new file mode 100644 index 000000000000..74ed7edd2736 --- /dev/null +++ b/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreSchemaTransformReadConfiguration.java @@ -0,0 +1,79 @@ +/* + * 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.singlestore; + +import com.google.auto.value.AutoValue; +import javax.annotation.Nullable; +import org.apache.beam.sdk.schemas.AutoValueSchema; +import org.apache.beam.sdk.schemas.annotations.DefaultSchema; +import org.apache.beam.sdk.transforms.SerializableFunction; +import org.apache.beam.sdk.values.Row; +import org.apache.beam.sdk.values.TypeDescriptor; + +/** + * Configuration for reading from SignleStoreDB. + * + *

This class is meant to be used with {@link SingleStoreSchemaTransformReadProvider}. + */ +@DefaultSchema(AutoValueSchema.class) +@AutoValue +public abstract class SingleStoreSchemaTransformReadConfiguration { + + /** Instantiates a {@link SingleStoreSchemaTransformReadConfiguration.Builder}. */ + public static Builder builder() { + return new AutoValue_SingleStoreSchemaTransformReadConfiguration.Builder(); + } + + private static final AutoValueSchema AUTO_VALUE_SCHEMA = new AutoValueSchema(); + private static final TypeDescriptor TYPE_DESCRIPTOR = + TypeDescriptor.of(SingleStoreSchemaTransformReadConfiguration.class); + private static final SerializableFunction + ROW_SERIALIZABLE_FUNCTION = AUTO_VALUE_SCHEMA.toRowFunction(TYPE_DESCRIPTOR); + + /** Serializes configuration to a {@link Row}. */ + Row toBeamRow() { + return ROW_SERIALIZABLE_FUNCTION.apply(this); + } + + @Nullable + public abstract SingleStoreIO.DataSourceConfiguration getDataSourceConfiguration(); + + @Nullable + public abstract String getQuery(); + + @Nullable + public abstract String getTable(); + + @Nullable + public abstract Boolean getOutputParallelization(); + + @AutoValue.Builder + public abstract static class Builder { + + public abstract Builder setDataSourceConfiguration(SingleStoreIO.DataSourceConfiguration value); + + public abstract Builder setTable(String value); + + public abstract Builder setQuery(String value); + + public abstract Builder setOutputParallelization(Boolean value); + + /** Builds the {@link SingleStoreSchemaTransformReadConfiguration} configuration. */ + public abstract SingleStoreSchemaTransformReadConfiguration build(); + } +} diff --git a/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreSchemaTransformReadProvider.java b/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreSchemaTransformReadProvider.java new file mode 100644 index 000000000000..b008beb3b1b0 --- /dev/null +++ b/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreSchemaTransformReadProvider.java @@ -0,0 +1,140 @@ +/* + * 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.singlestore; + +import java.util.Collections; +import java.util.List; +import org.apache.beam.sdk.schemas.Schema; +import org.apache.beam.sdk.schemas.transforms.SchemaTransform; +import org.apache.beam.sdk.schemas.transforms.TypedSchemaTransformProvider; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PCollectionRowTuple; +import org.apache.beam.sdk.values.Row; + +/** + * An implementation of {@link TypedSchemaTransformProvider} for SingleStoreDB read jobs configured + * using {@link SingleStoreSchemaTransformReadConfiguration}. + */ +public class SingleStoreSchemaTransformReadProvider + extends TypedSchemaTransformProvider { + + private static final String API = "singlestore"; + private static final String OUTPUT_TAG = "OUTPUT"; + + /** Returns the expected class of the configuration. */ + @Override + protected Class configurationClass() { + return SingleStoreSchemaTransformReadConfiguration.class; + } + + /** Returns the expected {@link SchemaTransform} of the configuration. */ + @Override + protected SchemaTransform from(SingleStoreSchemaTransformReadConfiguration configuration) { + return new SingleStoreReadSchemaTransform(configuration); + } + + /** Implementation of the {@link TypedSchemaTransformProvider} identifier method. */ + @Override + public String identifier() { + return String.format("%s:read", API); + } + + /** + * Implementation of the {@link TypedSchemaTransformProvider} inputCollectionNames method. Since + * no input is expected, this returns an empty list. + */ + @Override + public List inputCollectionNames() { + return Collections.emptyList(); + } + + /** + * Implementation of the {@link TypedSchemaTransformProvider} outputCollectionNames method. Since + * a single output is expected, this returns a list with a single name. + */ + @Override + public List outputCollectionNames() { + return Collections.singletonList(OUTPUT_TAG); + } + + /** + * An implementation of {@link SchemaTransform} for SingleStoreDB read jobs configured using + * {@link SingleStoreSchemaTransformReadConfiguration}. + */ + private static class SingleStoreReadSchemaTransform implements SchemaTransform { + private final SingleStoreSchemaTransformReadConfiguration configuration; + + SingleStoreReadSchemaTransform(SingleStoreSchemaTransformReadConfiguration configuration) { + this.configuration = configuration; + } + + /** Implements {@link SchemaTransform} buildTransform method. */ + @Override + public PTransform buildTransform() { + return new PCollectionRowTupleTransform(configuration); + } + } + + /** + * An implementation of {@link PTransform} for SingleStoreDB read jobs configured using {@link + * SingleStoreSchemaTransformReadConfiguration}. + */ + static class PCollectionRowTupleTransform + extends PTransform { + + private final SingleStoreSchemaTransformReadConfiguration configuration; + + PCollectionRowTupleTransform(SingleStoreSchemaTransformReadConfiguration configuration) { + this.configuration = configuration; + } + + @Override + public PCollectionRowTuple expand(PCollectionRowTuple input) { + if (!input.getAll().isEmpty()) { + throw new IllegalArgumentException( + String.format( + "%s %s input is expected to be empty", + input.getClass().getSimpleName(), getClass().getSimpleName())); + } + + SingleStoreIO.ReadRows read = SingleStoreIO.readRows(); + + if (configuration.getDataSourceConfiguration() != null) { + read = read.withDataSourceConfiguration(configuration.getDataSourceConfiguration()); + } + + if (configuration.getTable() != null) { + read = read.withTable(configuration.getTable()); + } + + if (configuration.getQuery() != null) { + read = read.withQuery(configuration.getQuery()); + } + + if (configuration.getOutputParallelization() != null) { + read = read.withOutputParallelization(configuration.getOutputParallelization()); + } + + PCollection rows = input.getPipeline().apply(read); + Schema schema = rows.getSchema(); + + return PCollectionRowTuple.of(OUTPUT_TAG, rows.setRowSchema(schema)); + } + } +} diff --git a/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreSchemaTransformReadWithPartitionsConfiguration.java b/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreSchemaTransformReadWithPartitionsConfiguration.java new file mode 100644 index 000000000000..2e0a89a3691f --- /dev/null +++ b/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreSchemaTransformReadWithPartitionsConfiguration.java @@ -0,0 +1,84 @@ +/* + * 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.singlestore; + +import com.google.auto.value.AutoValue; +import javax.annotation.Nullable; +import org.apache.beam.sdk.schemas.AutoValueSchema; +import org.apache.beam.sdk.schemas.annotations.DefaultSchema; +import org.apache.beam.sdk.transforms.SerializableFunction; +import org.apache.beam.sdk.values.Row; +import org.apache.beam.sdk.values.TypeDescriptor; + +/** + * Configuration for parallel reading from SignleStoreDB. + * + *

This class is meant to be used with {@link + * SingleStoreSchemaTransformReadWithPartitionsProvider}. + */ +@DefaultSchema(AutoValueSchema.class) +@AutoValue +public abstract class SingleStoreSchemaTransformReadWithPartitionsConfiguration { + + /** Instantiates a {@link SingleStoreSchemaTransformReadWithPartitionsConfiguration.Builder}. */ + public static Builder builder() { + return new AutoValue_SingleStoreSchemaTransformReadWithPartitionsConfiguration.Builder(); + } + + private static final AutoValueSchema AUTO_VALUE_SCHEMA = new AutoValueSchema(); + private static final TypeDescriptor + TYPE_DESCRIPTOR = + TypeDescriptor.of(SingleStoreSchemaTransformReadWithPartitionsConfiguration.class); + private static final SerializableFunction< + SingleStoreSchemaTransformReadWithPartitionsConfiguration, Row> + ROW_SERIALIZABLE_FUNCTION = AUTO_VALUE_SCHEMA.toRowFunction(TYPE_DESCRIPTOR); + + /** Serializes configuration to a {@link Row}. */ + Row toBeamRow() { + return ROW_SERIALIZABLE_FUNCTION.apply(this); + } + + @Nullable + public abstract SingleStoreIO.DataSourceConfiguration getDataSourceConfiguration(); + + @Nullable + public abstract String getQuery(); + + @Nullable + public abstract String getTable(); + + @Nullable + public abstract Integer getInitialNumReaders(); + + @AutoValue.Builder + public abstract static class Builder { + + public abstract Builder setDataSourceConfiguration(SingleStoreIO.DataSourceConfiguration value); + + public abstract Builder setTable(String value); + + public abstract Builder setQuery(String value); + + public abstract Builder setInitialNumReaders(Integer value); + + /** + * Builds the {@link SingleStoreSchemaTransformReadWithPartitionsConfiguration} configuration. + */ + public abstract SingleStoreSchemaTransformReadWithPartitionsConfiguration build(); + } +} diff --git a/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreSchemaTransformReadWithPartitionsProvider.java b/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreSchemaTransformReadWithPartitionsProvider.java new file mode 100644 index 000000000000..8d608165c9c3 --- /dev/null +++ b/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreSchemaTransformReadWithPartitionsProvider.java @@ -0,0 +1,144 @@ +/* + * 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.singlestore; + +import java.util.Collections; +import java.util.List; +import org.apache.beam.sdk.schemas.Schema; +import org.apache.beam.sdk.schemas.transforms.SchemaTransform; +import org.apache.beam.sdk.schemas.transforms.TypedSchemaTransformProvider; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PCollectionRowTuple; +import org.apache.beam.sdk.values.Row; + +/** + * An implementation of {@link TypedSchemaTransformProvider} for SingleStoreDB parallel read jobs + * configured using {@link SingleStoreSchemaTransformReadWithPartitionsConfiguration}. + */ +public class SingleStoreSchemaTransformReadWithPartitionsProvider + extends TypedSchemaTransformProvider< + SingleStoreSchemaTransformReadWithPartitionsConfiguration> { + + private static final String API = "singlestore"; + private static final String OUTPUT_TAG = "OUTPUT"; + + /** Returns the expected class of the configuration. */ + @Override + protected Class configurationClass() { + return SingleStoreSchemaTransformReadWithPartitionsConfiguration.class; + } + + /** Returns the expected {@link SchemaTransform} of the configuration. */ + @Override + protected SchemaTransform from( + SingleStoreSchemaTransformReadWithPartitionsConfiguration configuration) { + return new SingleStoreReadWithPartitionsSchemaTransform(configuration); + } + + /** Implementation of the {@link TypedSchemaTransformProvider} identifier method. */ + @Override + public String identifier() { + return String.format("%s:read-with-partitions", API); + } + + /** + * Implementation of the {@link TypedSchemaTransformProvider} inputCollectionNames method. Since + * no input is expected, this returns an empty list. + */ + @Override + public List inputCollectionNames() { + return Collections.emptyList(); + } + + /** + * Implementation of the {@link TypedSchemaTransformProvider} outputCollectionNames method. Since + * a single output is expected, this returns a list with a single name. + */ + @Override + public List outputCollectionNames() { + return Collections.singletonList(OUTPUT_TAG); + } + + /** + * An implementation of {@link SchemaTransform} for SingleStoreDB parallel read jobs configured + * using {@link SingleStoreSchemaTransformReadWithPartitionsConfiguration}. + */ + private static class SingleStoreReadWithPartitionsSchemaTransform implements SchemaTransform { + private final SingleStoreSchemaTransformReadWithPartitionsConfiguration configuration; + + SingleStoreReadWithPartitionsSchemaTransform( + SingleStoreSchemaTransformReadWithPartitionsConfiguration configuration) { + this.configuration = configuration; + } + + /** Implements {@link SchemaTransform} buildTransform method. */ + @Override + public PTransform buildTransform() { + return new PCollectionRowTupleTransform(configuration); + } + } + + /** + * An implementation of {@link PTransform} for SingleStoreDB read jobs configured using {@link + * SingleStoreSchemaTransformReadWithPartitionsConfiguration}. + */ + static class PCollectionRowTupleTransform + extends PTransform { + + private final SingleStoreSchemaTransformReadWithPartitionsConfiguration configuration; + + PCollectionRowTupleTransform( + SingleStoreSchemaTransformReadWithPartitionsConfiguration configuration) { + this.configuration = configuration; + } + + @Override + public PCollectionRowTuple expand(PCollectionRowTuple input) { + if (!input.getAll().isEmpty()) { + throw new IllegalArgumentException( + String.format( + "%s %s input is expected to be empty", + input.getClass().getSimpleName(), getClass().getSimpleName())); + } + + SingleStoreIO.ReadWithPartitionsRows read = SingleStoreIO.readWithPartitionsRows(); + + if (configuration.getDataSourceConfiguration() != null) { + read = read.withDataSourceConfiguration(configuration.getDataSourceConfiguration()); + } + + if (configuration.getTable() != null) { + read = read.withTable(configuration.getTable()); + } + + if (configuration.getQuery() != null) { + read = read.withQuery(configuration.getQuery()); + } + + if (configuration.getInitialNumReaders() != null) { + read = read.withInitialNumReaders(configuration.getInitialNumReaders()); + } + + PCollection rows = input.getPipeline().apply(read); + Schema schema = rows.getSchema(); + + return PCollectionRowTuple.of(OUTPUT_TAG, rows.setRowSchema(schema)); + } + } +} diff --git a/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreSchemaTransformWriteConfiguration.java b/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreSchemaTransformWriteConfiguration.java new file mode 100644 index 000000000000..c5694daf8045 --- /dev/null +++ b/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreSchemaTransformWriteConfiguration.java @@ -0,0 +1,74 @@ +/* + * 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.singlestore; + +import com.google.auto.value.AutoValue; +import javax.annotation.Nullable; +import org.apache.beam.sdk.schemas.AutoValueSchema; +import org.apache.beam.sdk.schemas.annotations.DefaultSchema; +import org.apache.beam.sdk.transforms.SerializableFunction; +import org.apache.beam.sdk.values.Row; +import org.apache.beam.sdk.values.TypeDescriptor; + +/** + * Configuration for writing to SignleStoreDB. + * + *

This class is meant to be used with {@link SingleStoreSchemaTransformWriteProvider}. + */ +@DefaultSchema(AutoValueSchema.class) +@AutoValue +public abstract class SingleStoreSchemaTransformWriteConfiguration { + + /** Instantiates a {@link SingleStoreSchemaTransformWriteConfiguration.Builder}. */ + public static Builder builder() { + return new AutoValue_SingleStoreSchemaTransformWriteConfiguration.Builder(); + } + + private static final AutoValueSchema AUTO_VALUE_SCHEMA = new AutoValueSchema(); + private static final TypeDescriptor + TYPE_DESCRIPTOR = TypeDescriptor.of(SingleStoreSchemaTransformWriteConfiguration.class); + private static final SerializableFunction + ROW_SERIALIZABLE_FUNCTION = AUTO_VALUE_SCHEMA.toRowFunction(TYPE_DESCRIPTOR); + + /** Serializes configuration to a {@link Row}. */ + Row toBeamRow() { + return ROW_SERIALIZABLE_FUNCTION.apply(this); + } + + @Nullable + public abstract SingleStoreIO.DataSourceConfiguration getDataSourceConfiguration(); + + @Nullable + public abstract String getTable(); + + @Nullable + public abstract Integer getBatchSize(); + + @AutoValue.Builder + public abstract static class Builder { + + public abstract Builder setDataSourceConfiguration(SingleStoreIO.DataSourceConfiguration value); + + public abstract Builder setTable(String value); + + public abstract Builder setBatchSize(Integer value); + + /** Builds the {@link SingleStoreSchemaTransformWriteConfiguration} configuration. */ + public abstract SingleStoreSchemaTransformWriteConfiguration build(); + } +} diff --git a/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreSchemaTransformWriteProvider.java b/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreSchemaTransformWriteProvider.java new file mode 100644 index 000000000000..49ffa01cd2e8 --- /dev/null +++ b/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreSchemaTransformWriteProvider.java @@ -0,0 +1,152 @@ +/* + * 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.singlestore; + +import java.util.Collections; +import java.util.List; +import org.apache.beam.sdk.schemas.Schema; +import org.apache.beam.sdk.schemas.transforms.SchemaTransform; +import org.apache.beam.sdk.schemas.transforms.TypedSchemaTransformProvider; +import org.apache.beam.sdk.transforms.MapElements; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.transforms.SerializableFunction; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PCollectionRowTuple; +import org.apache.beam.sdk.values.Row; +import org.apache.beam.sdk.values.TypeDescriptor; + +/** + * An implementation of {@link TypedSchemaTransformProvider} for SingleStoreDB write jobs configured + * using {@link SingleStoreSchemaTransformWriteConfiguration}. + */ +public class SingleStoreSchemaTransformWriteProvider + extends TypedSchemaTransformProvider { + + private static final String API = "singlestore"; + private static final String OUTPUT_TAG = "OUTPUT"; + static final String INPUT_TAG = "INPUT"; + + /** Returns the expected class of the configuration. */ + @Override + protected Class configurationClass() { + return SingleStoreSchemaTransformWriteConfiguration.class; + } + + /** Returns the expected {@link SchemaTransform} of the configuration. */ + @Override + protected SchemaTransform from(SingleStoreSchemaTransformWriteConfiguration configuration) { + return new SingleStoreWriteSchemaTransform(configuration); + } + + /** Implementation of the {@link TypedSchemaTransformProvider} identifier method. */ + @Override + public String identifier() { + return String.format("%s:write", API); + } + + /** + * Implementation of the {@link TypedSchemaTransformProvider} inputCollectionNames method. Since + * no input is expected, this returns an empty list. + */ + @Override + public List inputCollectionNames() { + return Collections.emptyList(); + } + + /** + * Implementation of the {@link TypedSchemaTransformProvider} outputCollectionNames method. Since + * a single output is expected, this returns a list with a single name. + */ + @Override + public List outputCollectionNames() { + return Collections.singletonList(OUTPUT_TAG); + } + + /** + * An implementation of {@link SchemaTransform} for SingleStoreDB write jobs configured using + * {@link SingleStoreSchemaTransformWriteConfiguration}. + */ + private static class SingleStoreWriteSchemaTransform implements SchemaTransform { + private final SingleStoreSchemaTransformWriteConfiguration configuration; + + SingleStoreWriteSchemaTransform(SingleStoreSchemaTransformWriteConfiguration configuration) { + this.configuration = configuration; + } + + /** Implements {@link SchemaTransform} buildTransform method. */ + @Override + public PTransform buildTransform() { + return new PCollectionRowTupleTransform(configuration); + } + } + + /** + * An implementation of {@link PTransform} for SingleStoreDB write jobs configured using {@link + * SingleStoreSchemaTransformWriteConfiguration}. + */ + static class PCollectionRowTupleTransform + extends PTransform { + + private final SingleStoreSchemaTransformWriteConfiguration configuration; + + PCollectionRowTupleTransform(SingleStoreSchemaTransformWriteConfiguration configuration) { + this.configuration = configuration; + } + + @Override + public PCollectionRowTuple expand(PCollectionRowTuple input) { + if (!input.has(INPUT_TAG)) { + throw new IllegalArgumentException( + String.format( + "%s %s is missing expected tag: %s", + getClass().getSimpleName(), input.getClass().getSimpleName(), INPUT_TAG)); + } + + SingleStoreIO.Write write = SingleStoreIO.writeRows(); + + if (configuration.getDataSourceConfiguration() != null) { + write = write.withDataSourceConfiguration(configuration.getDataSourceConfiguration()); + } + + if (configuration.getTable() != null) { + write = write.withTable(configuration.getTable()); + } + + if (configuration.getBatchSize() != null) { + write = write.withBatchSize(configuration.getBatchSize()); + } + + PCollection res = input.get(INPUT_TAG).apply(write); + Schema.Builder schemaBuilder = new Schema.Builder(); + schemaBuilder.addField("rowsWritten", Schema.FieldType.INT32); + Schema schema = schemaBuilder.build(); + return PCollectionRowTuple.of( + OUTPUT_TAG, + res.apply( + MapElements.into(TypeDescriptor.of(Row.class)) + .via( + (SerializableFunction) + a -> { + Row.Builder rowBuilder = Row.withSchema(schema); + rowBuilder.addValue(a); + return rowBuilder.build(); + })) + .setRowSchema(schema)); + } + } +} diff --git a/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreUtil.java b/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreUtil.java index c76b0a678c15..3b04cb9f8c0a 100644 --- a/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreUtil.java +++ b/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreUtil.java @@ -38,10 +38,15 @@ public static String escapeString(String identifier) { } public static Coder inferCoder( + @Nullable Coder coder, SingleStoreIO.RowMapper rowMapper, CoderRegistry registry, SchemaRegistry schemaRegistry, Logger log) { + if (coder != null) { + return coder; + } + TypeDescriptor outputType = TypeDescriptors.extractFromTypeParameters( rowMapper, diff --git a/sdks/java/io/singlestore/src/test/java/org/apache/beam/sdk/io/singlestore/SingleStoreDefaultRowMapperTest.java b/sdks/java/io/singlestore/src/test/java/org/apache/beam/sdk/io/singlestore/SingleStoreDefaultRowMapperTest.java new file mode 100644 index 000000000000..bf5f8f2113e3 --- /dev/null +++ b/sdks/java/io/singlestore/src/test/java/org/apache/beam/sdk/io/singlestore/SingleStoreDefaultRowMapperTest.java @@ -0,0 +1,235 @@ +/* + * 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.singlestore; + +import static java.sql.Types.*; +import static org.junit.Assert.assertArrayEquals; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; + +import com.singlestore.jdbc.client.result.ResultSetMetaData; +import java.math.BigDecimal; +import java.nio.charset.StandardCharsets; +import java.sql.ResultSet; +import java.sql.Time; +import java.sql.Timestamp; +import java.time.LocalDate; +import org.apache.beam.sdk.schemas.Schema; +import org.apache.beam.sdk.values.Row; +import org.joda.time.DateTime; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; +import org.mockito.Mockito; + +/** Test DefaultRowMapper. */ +@RunWith(JUnit4.class) +public class SingleStoreDefaultRowMapperTest { + @Test + public void testEmptyRow() throws Exception { + ResultSetMetaData md = Mockito.mock(ResultSetMetaData.class); + Mockito.when(md.getColumnCount()).thenReturn(0); + ResultSet res = Mockito.mock(ResultSet.class); + + SingleStoreDefaultRowMapper mapper = new SingleStoreDefaultRowMapper(md); + Schema s = mapper.getSchema(); + Row r = mapper.mapRow(res); + + assertEquals(0, s.getFieldCount()); + assertEquals(0, r.getFieldCount()); + } + + @Test + public void testAllDataTypes() throws Exception { + ResultSetMetaData md = Mockito.mock(ResultSetMetaData.class); + + Mockito.when(md.getColumnCount()).thenReturn(17); + + Mockito.when(md.getColumnType(1)).thenReturn(BIT); + Mockito.when(md.getColumnType(2)).thenReturn(TINYINT); + Mockito.when(md.getColumnType(3)).thenReturn(SMALLINT); + Mockito.when(md.getColumnType(4)).thenReturn(INTEGER); + Mockito.when(md.getColumnType(5)).thenReturn(BIGINT); + Mockito.when(md.getColumnType(6)).thenReturn(REAL); + Mockito.when(md.getColumnType(7)).thenReturn(DOUBLE); + Mockito.when(md.getColumnType(8)).thenReturn(DECIMAL); + Mockito.when(md.getColumnType(9)).thenReturn(TIMESTAMP); + Mockito.when(md.getColumnType(10)).thenReturn(DATE); + Mockito.when(md.getColumnType(11)).thenReturn(TIME); + Mockito.when(md.getColumnType(12)).thenReturn(LONGVARBINARY); + Mockito.when(md.getColumnType(13)).thenReturn(VARBINARY); + Mockito.when(md.getColumnType(14)).thenReturn(BINARY); + Mockito.when(md.getColumnType(15)).thenReturn(LONGVARCHAR); + Mockito.when(md.getColumnType(16)).thenReturn(VARCHAR); + Mockito.when(md.getColumnType(17)).thenReturn(CHAR); + + for (int i = 12; i <= 17; i++) { + Mockito.when(md.getPrecision(i)).thenReturn(10); + } + + for (int i = 1; i <= md.getColumnCount(); i++) { + Mockito.when(md.getColumnLabel(i)).thenReturn("c" + i); + Mockito.when(md.isNullable(i)).thenReturn(java.sql.ResultSetMetaData.columnNoNulls); + } + + ResultSet res = Mockito.mock(ResultSet.class); + + Mockito.when(res.getBoolean(1)).thenReturn(true); + Mockito.when(res.getByte(2)).thenReturn((byte) 10); + Mockito.when(res.getShort(3)).thenReturn((short) 10); + Mockito.when(res.getInt(4)).thenReturn(10); + Mockito.when(res.getLong(5)).thenReturn((long) 10); + Mockito.when(res.getFloat(6)).thenReturn((float) 10.1); + Mockito.when(res.getDouble(7)).thenReturn(10.1); + Mockito.when(res.getBigDecimal(8)).thenReturn(new BigDecimal("100.100")); + Mockito.when(res.getTimestamp(Mockito.eq(9), Mockito.any())) + .thenReturn(Timestamp.valueOf("2022-10-10 10:10:10")); + Mockito.when(res.getObject(10, java.time.LocalDate.class)) + .thenReturn(LocalDate.of(2022, 10, 10)); + Mockito.when(res.getTime(Mockito.eq(11), Mockito.any())).thenReturn(Time.valueOf("10:10:10")); + Mockito.when(res.getBytes(12)).thenReturn("asd".getBytes(StandardCharsets.UTF_8)); + Mockito.when(res.getBytes(13)).thenReturn("asd".getBytes(StandardCharsets.UTF_8)); + Mockito.when(res.getBytes(14)).thenReturn("asd\0\0\0\0\0\0\0".getBytes(StandardCharsets.UTF_8)); + Mockito.when(res.getString(15)).thenReturn("asd"); + Mockito.when(res.getString(16)).thenReturn("asd"); + Mockito.when(res.getString(17)).thenReturn("asd"); + + Mockito.when(res.wasNull()).thenReturn(false); + + SingleStoreDefaultRowMapper mapper = new SingleStoreDefaultRowMapper(md); + Schema s = mapper.getSchema(); + Row r = mapper.mapRow(res); + + assertEquals(17, s.getFieldCount()); + for (int i = 0; i < s.getFieldCount(); i++) { + assertEquals("c" + (i + 1), s.getField(i).getName()); + } + + assertEquals(Schema.FieldType.BOOLEAN, s.getField(0).getType()); + assertEquals(Schema.FieldType.BYTE, s.getField(1).getType()); + assertEquals(Schema.FieldType.INT16, s.getField(2).getType()); + assertEquals(Schema.FieldType.INT32, s.getField(3).getType()); + assertEquals(Schema.FieldType.INT64, s.getField(4).getType()); + assertEquals(Schema.FieldType.FLOAT, s.getField(5).getType()); + assertEquals(Schema.FieldType.DOUBLE, s.getField(6).getType()); + assertEquals(Schema.FieldType.DECIMAL, s.getField(7).getType()); + assertEquals(Schema.FieldType.DATETIME, s.getField(8).getType()); + assertEquals(Schema.FieldType.DATETIME, s.getField(9).getType()); + assertEquals(Schema.FieldType.DATETIME, s.getField(10).getType()); + assertEquals(Schema.TypeName.LOGICAL_TYPE, s.getField(11).getType().getTypeName()); + assertEquals(Schema.TypeName.LOGICAL_TYPE, s.getField(12).getType().getTypeName()); + assertEquals(Schema.TypeName.LOGICAL_TYPE, s.getField(13).getType().getTypeName()); + assertEquals(Schema.TypeName.LOGICAL_TYPE, s.getField(14).getType().getTypeName()); + assertEquals(Schema.TypeName.LOGICAL_TYPE, s.getField(15).getType().getTypeName()); + assertEquals(Schema.TypeName.LOGICAL_TYPE, s.getField(16).getType().getTypeName()); + assertEquals(Schema.FieldType.BYTES, s.getField(11).getType().getLogicalType().getBaseType()); + assertEquals(Schema.FieldType.BYTES, s.getField(12).getType().getLogicalType().getBaseType()); + assertEquals(Schema.FieldType.BYTES, s.getField(13).getType().getLogicalType().getBaseType()); + assertEquals(Schema.FieldType.STRING, s.getField(14).getType().getLogicalType().getBaseType()); + assertEquals(Schema.FieldType.STRING, s.getField(15).getType().getLogicalType().getBaseType()); + assertEquals(Schema.FieldType.STRING, s.getField(16).getType().getLogicalType().getBaseType()); + + assertEquals(17, r.getFieldCount()); + + assertEquals(true, r.getBoolean(0)); + assertEquals((Byte) (byte) 10, r.getByte(1)); + assertEquals((Short) (short) 10, r.getInt16(2)); + assertEquals((Integer) 10, r.getInt32(3)); + assertEquals((Long) (long) 10, r.getInt64(4)); + assertEquals((Float) (float) 10.1, r.getFloat(5)); + assertEquals((Double) 10.1, r.getDouble(6)); + assertEquals(new BigDecimal("100.100"), r.getDecimal(7)); + assertEquals(0, new DateTime("2022-10-10T10:10:10").compareTo(r.getDateTime(8))); + assertEquals(0, new DateTime("2022-10-10T00:00:00Z").compareTo(r.getDateTime(9))); + assertEquals(0, new DateTime("1970-01-01T10:10:10").compareTo(r.getDateTime(10))); + assertArrayEquals("asd".getBytes(StandardCharsets.UTF_8), r.getBytes(11)); + assertArrayEquals("asd".getBytes(StandardCharsets.UTF_8), r.getBytes(12)); + assertArrayEquals("asd\0\0\0\0\0\0\0".getBytes(StandardCharsets.UTF_8), r.getBytes(13)); + assertEquals("asd", r.getString(14)); + assertEquals("asd", r.getString(15)); + assertEquals("asd", r.getString(16)); + } + + @Test + public void testNullValues() throws Exception { + ResultSetMetaData md = Mockito.mock(ResultSetMetaData.class); + + Mockito.when(md.getColumnCount()).thenReturn(18); + + Mockito.when(md.getColumnType(1)).thenReturn(BIT); + Mockito.when(md.getColumnType(2)).thenReturn(TINYINT); + Mockito.when(md.getColumnType(3)).thenReturn(SMALLINT); + Mockito.when(md.getColumnType(4)).thenReturn(INTEGER); + Mockito.when(md.getColumnType(5)).thenReturn(BIGINT); + Mockito.when(md.getColumnType(6)).thenReturn(REAL); + Mockito.when(md.getColumnType(7)).thenReturn(DOUBLE); + Mockito.when(md.getColumnType(8)).thenReturn(DECIMAL); + Mockito.when(md.getColumnType(9)).thenReturn(TIMESTAMP); + Mockito.when(md.getColumnType(10)).thenReturn(DATE); + Mockito.when(md.getColumnType(11)).thenReturn(TIME); + Mockito.when(md.getColumnType(12)).thenReturn(LONGVARBINARY); + Mockito.when(md.getColumnType(13)).thenReturn(VARBINARY); + Mockito.when(md.getColumnType(14)).thenReturn(BINARY); + Mockito.when(md.getColumnType(15)).thenReturn(LONGVARCHAR); + Mockito.when(md.getColumnType(16)).thenReturn(VARCHAR); + Mockito.when(md.getColumnType(17)).thenReturn(CHAR); + Mockito.when(md.getColumnType(18)).thenReturn(NULL); + + for (int i = 1; i <= md.getColumnCount(); i++) { + Mockito.when(md.getColumnLabel(i)).thenReturn("c" + i); + Mockito.when(md.isNullable(i)).thenReturn(java.sql.ResultSetMetaData.columnNullable); + } + + ResultSet res = Mockito.mock(ResultSet.class); + + Mockito.when(res.getBoolean(1)).thenReturn(false); + Mockito.when(res.getByte(2)).thenReturn((byte) 0); + Mockito.when(res.getShort(3)).thenReturn((short) 0); + Mockito.when(res.getInt(4)).thenReturn(0); + Mockito.when(res.getLong(5)).thenReturn((long) 0); + Mockito.when(res.getFloat(6)).thenReturn((float) 0); + Mockito.when(res.getDouble(7)).thenReturn(0.0); + Mockito.when(res.getBigDecimal(8)).thenReturn(null); + Mockito.when(res.getTimestamp(9)).thenReturn(null); + Mockito.when(res.getDate(10)).thenReturn(null); + Mockito.when(res.getTime(11)).thenReturn(null); + Mockito.when(res.getBytes(12)).thenReturn(null); + Mockito.when(res.getBytes(13)).thenReturn(null); + Mockito.when(res.getBytes(14)).thenReturn(null); + Mockito.when(res.getString(15)).thenReturn(null); + Mockito.when(res.getString(16)).thenReturn(null); + Mockito.when(res.getString(17)).thenReturn(null); + Mockito.when(res.getString(18)).thenReturn(null); + + Mockito.when(res.wasNull()).thenReturn(true); + + SingleStoreDefaultRowMapper mapper = new SingleStoreDefaultRowMapper(md); + Schema s = mapper.getSchema(); + Row r = mapper.mapRow(res); + + assertEquals(18, s.getFieldCount()); + for (int i = 0; i < s.getFieldCount(); i++) { + assertEquals("c" + (i + 1), s.getField(i).getName()); + } + + assertEquals(18, r.getFieldCount()); + for (int i = 0; i < r.getFieldCount(); i++) { + assertNull(r.getValue(i)); + } + } +} diff --git a/sdks/java/io/singlestore/src/test/java/org/apache/beam/sdk/io/singlestore/SingleStoreDefaultUserDataMapperTest.java b/sdks/java/io/singlestore/src/test/java/org/apache/beam/sdk/io/singlestore/SingleStoreDefaultUserDataMapperTest.java new file mode 100644 index 000000000000..a6b52e2f4d4f --- /dev/null +++ b/sdks/java/io/singlestore/src/test/java/org/apache/beam/sdk/io/singlestore/SingleStoreDefaultUserDataMapperTest.java @@ -0,0 +1,136 @@ +/* + * 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.singlestore; + +import static org.junit.Assert.assertEquals; + +import java.math.BigDecimal; +import java.nio.charset.StandardCharsets; +import java.util.List; +import org.apache.beam.sdk.schemas.Schema; +import org.apache.beam.sdk.values.Row; +import org.joda.time.DateTime; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +/** Test DefaultUserDataMapper. */ +@RunWith(JUnit4.class) +public class SingleStoreDefaultUserDataMapperTest { + @Test + public void testNullValues() {} + + @Test + public void testBigNumbers() {} + + @Test + public void testBigNegativeNumbers() { + Schema.Builder schemaBuilder = new Schema.Builder(); + schemaBuilder.addField("byte", Schema.FieldType.BYTE); + schemaBuilder.addField("int16", Schema.FieldType.INT16); + schemaBuilder.addField("int32", Schema.FieldType.INT32); + schemaBuilder.addField("int64", Schema.FieldType.INT64); + schemaBuilder.addField("float", Schema.FieldType.FLOAT); + schemaBuilder.addField("double", Schema.FieldType.DOUBLE); + schemaBuilder.addField("decimal", Schema.FieldType.DECIMAL); + Schema schema = schemaBuilder.build(); + + Row.Builder rowBuilder = Row.withSchema(schema); + rowBuilder.addValue(Byte.MIN_VALUE); + rowBuilder.addValue(Short.MIN_VALUE); + rowBuilder.addValue(Integer.MIN_VALUE); + rowBuilder.addValue(Long.MIN_VALUE); + rowBuilder.addValue(-Float.MAX_VALUE); + rowBuilder.addValue(-Double.MAX_VALUE); + rowBuilder.addValue(new BigDecimal("-10000000000000.1000000000000000000000")); + Row row = rowBuilder.build(); + + SingleStoreDefaultUserDataMapper mapper = new SingleStoreDefaultUserDataMapper(); + List res = mapper.mapRow(row); + + assertEquals(7, res.size()); + assertEquals("-128", res.get(0)); + assertEquals("-32768", res.get(1)); + assertEquals("10", res.get(2)); + assertEquals("10", res.get(3)); + assertEquals("10.1", res.get(4)); + assertEquals("10.1", res.get(5)); + assertEquals("-10000000000000.1000000000000000000000", res.get(6)); + } + + @Test + public void testEmptyRow() { + Schema.Builder schemaBuilder = new Schema.Builder(); + Schema schema = schemaBuilder.build(); + + Row.Builder rowBuilder = Row.withSchema(schema); + Row row = rowBuilder.build(); + + SingleStoreDefaultUserDataMapper mapper = new SingleStoreDefaultUserDataMapper(); + List res = mapper.mapRow(row); + + assertEquals(0, res.size()); + } + + @Test + public void testAllDataTypes() { + Schema.Builder schemaBuilder = new Schema.Builder(); + schemaBuilder.addField("byte", Schema.FieldType.BYTE); + schemaBuilder.addField("int16", Schema.FieldType.INT16); + schemaBuilder.addField("int32", Schema.FieldType.INT32); + schemaBuilder.addField("int64", Schema.FieldType.INT64); + schemaBuilder.addField("float", Schema.FieldType.FLOAT); + schemaBuilder.addField("double", Schema.FieldType.DOUBLE); + schemaBuilder.addField("decimal", Schema.FieldType.DECIMAL); + schemaBuilder.addField("boolean", Schema.FieldType.BOOLEAN); + schemaBuilder.addField("datetime", Schema.FieldType.DATETIME); + schemaBuilder.addField("bytes", Schema.FieldType.BYTES); + schemaBuilder.addField("string", Schema.FieldType.STRING); + Schema schema = schemaBuilder.build(); + + Row.Builder rowBuilder = Row.withSchema(schema); + rowBuilder.addValue((byte) 10); + rowBuilder.addValue((short) 10); + rowBuilder.addValue(10); + rowBuilder.addValue((long) 10); + rowBuilder.addValue((float) 10.1); + rowBuilder.addValue(10.1); + rowBuilder.addValue(new BigDecimal("10.1")); + rowBuilder.addValue(false); + rowBuilder.addValue(new DateTime("2022-01-01T10:10:10.012Z")); + rowBuilder.addValue("asd".getBytes(StandardCharsets.UTF_8)); + rowBuilder.addValue("asd"); + Row row = rowBuilder.build(); + + SingleStoreDefaultUserDataMapper mapper = new SingleStoreDefaultUserDataMapper(); + List res = mapper.mapRow(row); + + assertEquals(11, res.size()); + assertEquals("10", res.get(0)); + assertEquals("10", res.get(1)); + assertEquals("10", res.get(2)); + assertEquals("10", res.get(3)); + assertEquals("10.1", res.get(4)); + assertEquals("10.1", res.get(5)); + assertEquals("10.1", res.get(6)); + assertEquals("0", res.get(7)); + assertEquals("2022-01-01 10:10:10.012", res.get(8)); + assertEquals("asd", res.get(9)); + assertEquals("asd", res.get(10)); + } +} diff --git a/sdks/java/io/singlestore/src/test/java/org/apache/beam/sdk/io/singlestore/SingleStoreIODefaultMapperIT.java b/sdks/java/io/singlestore/src/test/java/org/apache/beam/sdk/io/singlestore/SingleStoreIODefaultMapperIT.java new file mode 100644 index 000000000000..32fc9d1a11f4 --- /dev/null +++ b/sdks/java/io/singlestore/src/test/java/org/apache/beam/sdk/io/singlestore/SingleStoreIODefaultMapperIT.java @@ -0,0 +1,252 @@ +/* + * 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.singlestore; + +import static org.apache.beam.sdk.io.common.IOITHelper.readIOTestPipelineOptions; +import static org.junit.Assert.assertArrayEquals; +import static org.junit.Assert.assertEquals; + +import java.math.BigDecimal; +import java.nio.charset.StandardCharsets; +import java.sql.Connection; +import java.sql.SQLException; +import java.sql.Statement; +import javax.sql.DataSource; +import org.apache.beam.sdk.PipelineResult; +import org.apache.beam.sdk.coders.VoidCoder; +import org.apache.beam.sdk.io.GenerateSequence; +import org.apache.beam.sdk.io.common.DatabaseTestHelper; +import org.apache.beam.sdk.schemas.Schema; +import org.apache.beam.sdk.testing.NeedsRunner; +import org.apache.beam.sdk.testing.PAssert; +import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.transforms.Count; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.Row; +import org.joda.time.DateTime; +import org.junit.Assert; +import org.junit.BeforeClass; +import org.junit.Rule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +@RunWith(JUnit4.class) +public class SingleStoreIODefaultMapperIT { + private static final String DATABASE_NAME = "SingleStoreIOIT"; + + private static int numberOfRows; + + private static String tableName; + + private static String serverName; + + private static String username; + + private static String password; + + private static Integer port; + + private static SingleStoreIO.DataSourceConfiguration dataSourceConfiguration; + + private static Schema schema; + + @BeforeClass + public static void setup() { + SingleStoreIOTestPipelineOptions options; + try { + options = readIOTestPipelineOptions(SingleStoreIOTestPipelineOptions.class); + } catch (IllegalArgumentException e) { + options = null; + } + org.junit.Assume.assumeNotNull(options); + + numberOfRows = options.getNumberOfRecords(); + serverName = options.getSingleStoreServerName(); + username = options.getSingleStoreUsername(); + password = options.getSingleStorePassword(); + port = options.getSingleStorePort(); + tableName = DatabaseTestHelper.getTestTableName("IT"); + dataSourceConfiguration = + SingleStoreIO.DataSourceConfiguration.create(serverName + ":" + port) + .withDatabase(DATABASE_NAME) + .withPassword(password) + .withUsername(username); + + generateSchema(); + } + + private static void generateSchema() { + Schema.Builder schemaBuilder = new Schema.Builder(); + schemaBuilder.addField("c1", Schema.FieldType.BOOLEAN); + schemaBuilder.addField("c2", Schema.FieldType.BYTE); + schemaBuilder.addField("c3", Schema.FieldType.INT16); + schemaBuilder.addField("c4", Schema.FieldType.INT32); + schemaBuilder.addField("c5", Schema.FieldType.INT64); + schemaBuilder.addField("c6", Schema.FieldType.FLOAT); + schemaBuilder.addField("c7", Schema.FieldType.DOUBLE); + schemaBuilder.addField("c8", Schema.FieldType.DECIMAL); + schemaBuilder.addField("c9", Schema.FieldType.DATETIME); + schemaBuilder.addField("c10", Schema.FieldType.DATETIME); + schemaBuilder.addField("c11", Schema.FieldType.DATETIME); + schemaBuilder.addField("c12", Schema.FieldType.BYTES); + schemaBuilder.addField("c13", Schema.FieldType.BYTES); + schemaBuilder.addField("c14", Schema.FieldType.BYTES); + schemaBuilder.addField("c15", Schema.FieldType.STRING); + schemaBuilder.addField("c16", Schema.FieldType.STRING); + schemaBuilder.addField("c17", Schema.FieldType.STRING); + + schema = schemaBuilder.build(); + } + + @Test + @Category(NeedsRunner.class) + public void testWriteThenRead() throws Exception { + TestHelper.createDatabaseIfNotExists(serverName, port, username, password, DATABASE_NAME); + createTable(); + + try { + PipelineResult writeResult = runWrite(); + assertEquals(PipelineResult.State.DONE, writeResult.waitUntilFinish()); + PipelineResult readResult = runRead(); + assertEquals(PipelineResult.State.DONE, readResult.waitUntilFinish()); + } finally { + DataSource dataSource = dataSourceConfiguration.getDataSource(); + DatabaseTestHelper.deleteTable(dataSource, tableName); + } + } + + private void createTable() throws SQLException { + DataSource dataSource = dataSourceConfiguration.getDataSource(); + Connection conn = dataSource.getConnection(); + try { + Statement stmt = conn.createStatement(); + stmt.executeUpdate("DROP TABLE IF EXISTS " + tableName); + stmt.executeUpdate( + "CREATE TABLE " + + tableName + + "(" + + "c1 BIT, " + + "c2 TINYINT, " + + "c3 SMALLINT, " + + "c4 INTEGER, " + + "c5 BIGINT, " + + "c6 FLOAT, " + + "c7 DOUBLE, " + + "c8 DECIMAL(10, 5), " + + "c9 TIMESTAMP, " + + "c10 DATE, " + + "c11 TIME, " + + "c12 BLOB, " + + "c13 TINYBLOB, " + + "c14 BINARY(10), " + + "c15 MEDIUMTEXT, " + + "c16 TINYTEXT, " + + "c17 CHAR(10) " + + ")"); + } finally { + conn.close(); + } + } + + @Rule public TestPipeline pipelineWrite = TestPipeline.create(); + @Rule public TestPipeline pipelineRead = TestPipeline.create(); + + public static class ConstructRowFn extends DoFn { + @ProcessElement + public void processElement(ProcessContext c) { + + Row.Builder rowBuilder = Row.withSchema(schema); + rowBuilder.addValue(Boolean.TRUE); + rowBuilder.addValue((byte) 10); + rowBuilder.addValue((short) 10); + rowBuilder.addValue(10); + rowBuilder.addValue((long) 10); + rowBuilder.addValue((float) 10.1); + rowBuilder.addValue(10.1); + rowBuilder.addValue(new BigDecimal("10.1")); + rowBuilder.addValue(new DateTime("2022-01-01T10:10:10Z")); + rowBuilder.addValue(new DateTime("2022-01-01T10:10:10Z")); + rowBuilder.addValue(new DateTime("2022-01-01T10:10:10Z")); + rowBuilder.addValue("asd".getBytes(StandardCharsets.UTF_8)); + rowBuilder.addValue("asd".getBytes(StandardCharsets.UTF_8)); + rowBuilder.addValue("asd".getBytes(StandardCharsets.UTF_8)); + rowBuilder.addValue("asd"); + rowBuilder.addValue("asd"); + rowBuilder.addValue("asd"); + Row row = rowBuilder.build(); + + c.output(row); + } + } + + private PipelineResult runWrite() { + pipelineWrite + .apply(GenerateSequence.from(0).to(numberOfRows)) + .apply(ParDo.of(new ConstructRowFn())) + .setRowSchema(schema) + .apply( + SingleStoreIO.writeRows() + .withDataSourceConfiguration(dataSourceConfiguration) + .withTable(tableName)); + + return pipelineWrite.run(); + } + + private PipelineResult runRead() { + PCollection res = + pipelineRead.apply( + SingleStoreIO.readRows() + .withDataSourceConfiguration(dataSourceConfiguration) + .withTable(tableName)); + + PAssert.thatSingleton(res.apply("Count All", Count.globally())).isEqualTo((long) numberOfRows); + + res.apply(ParDo.of(new CheckDoFn())).setCoder(VoidCoder.of()); + + return pipelineRead.run(); + } + + private static class CheckDoFn extends DoFn { + @ProcessElement + public void process(ProcessContext c) { + Row r = c.element(); + Assert.assertNotNull(r); + assertEquals(Boolean.TRUE, r.getBoolean(0)); + assertEquals((Byte) (byte) 10, r.getByte(1)); + assertEquals((Short) (short) 10, r.getInt16(2)); + assertEquals((Integer) 10, r.getInt32(3)); + assertEquals((Long) (long) 10, r.getInt64(4)); + assertEquals((Float) (float) 10.1, r.getFloat(5)); + assertEquals((Double) 10.1, r.getDouble(6)); + assertEquals(new BigDecimal("10.10000"), r.getDecimal(7)); + assertEquals(0, new DateTime("2022-01-01T10:10:10Z").compareTo(r.getDateTime(8))); + assertEquals(0, new DateTime("2022-01-01T00:00:00Z").compareTo(r.getDateTime(9))); + assertEquals(0, new DateTime("1970-01-01T10:10:10Z").compareTo(r.getDateTime(10))); + assertArrayEquals("asd".getBytes(StandardCharsets.UTF_8), r.getBytes(11)); + assertArrayEquals("asd".getBytes(StandardCharsets.UTF_8), r.getBytes(12)); + assertArrayEquals("asd\0\0\0\0\0\0\0".getBytes(StandardCharsets.UTF_8), r.getBytes(13)); + assertEquals("asd", r.getString(14)); + assertEquals("asd", r.getString(15)); + assertEquals("asd", r.getString(16)); + } + } +} diff --git a/sdks/java/io/singlestore/src/test/java/org/apache/beam/sdk/io/singlestore/SingleStoreIOIT.java b/sdks/java/io/singlestore/src/test/java/org/apache/beam/sdk/io/singlestore/SingleStoreIOPerformanceIT.java similarity index 92% rename from sdks/java/io/singlestore/src/test/java/org/apache/beam/sdk/io/singlestore/SingleStoreIOIT.java rename to sdks/java/io/singlestore/src/test/java/org/apache/beam/sdk/io/singlestore/SingleStoreIOPerformanceIT.java index cdbcacd04434..8adf8a47dc13 100644 --- a/sdks/java/io/singlestore/src/test/java/org/apache/beam/sdk/io/singlestore/SingleStoreIOIT.java +++ b/sdks/java/io/singlestore/src/test/java/org/apache/beam/sdk/io/singlestore/SingleStoreIOPerformanceIT.java @@ -22,9 +22,6 @@ import com.google.cloud.Timestamp; import com.singlestore.jdbc.SingleStoreDataSource; -import java.sql.Connection; -import java.sql.SQLException; -import java.sql.Statement; import java.util.HashSet; import java.util.List; import java.util.Set; @@ -58,9 +55,9 @@ import org.junit.runners.JUnit4; @RunWith(JUnit4.class) -public class SingleStoreIOIT { +public class SingleStoreIOPerformanceIT { - private static final String NAMESPACE = SingleStoreIOIT.class.getName(); + private static final String NAMESPACE = SingleStoreIOPerformanceIT.class.getName(); private static final String DATABASE_NAME = "SingleStoreIOIT"; @@ -109,22 +106,10 @@ public static void setup() { .get(); } - void createDatabaseIfNotExists() throws SQLException { - DataSource dataSource = - new SingleStoreDataSource( - String.format( - "jdbc:singlestore://%s:%d/?user=%s&password=%s&allowLocalInfile=TRUE", - serverName, port, username, password)); - try (Connection conn = dataSource.getConnection(); - Statement stmt = conn.createStatement()) { - stmt.executeQuery(String.format("CREATE DATABASE IF NOT EXISTS %s", DATABASE_NAME)); - } - } - @Test @Category(NeedsRunner.class) public void testWriteThenRead() throws Exception { - createDatabaseIfNotExists(); + TestHelper.createDatabaseIfNotExists(serverName, port, username, password, DATABASE_NAME); DataSource dataSource = new SingleStoreDataSource( String.format( diff --git a/sdks/java/io/singlestore/src/test/java/org/apache/beam/sdk/io/singlestore/SingleStoreIOSchemaTransformIT.java b/sdks/java/io/singlestore/src/test/java/org/apache/beam/sdk/io/singlestore/SingleStoreIOSchemaTransformIT.java new file mode 100644 index 000000000000..8542e0631b41 --- /dev/null +++ b/sdks/java/io/singlestore/src/test/java/org/apache/beam/sdk/io/singlestore/SingleStoreIOSchemaTransformIT.java @@ -0,0 +1,267 @@ +/* + * 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.singlestore; + +import static org.apache.beam.sdk.io.common.IOITHelper.readIOTestPipelineOptions; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +import com.singlestore.jdbc.SingleStoreDataSource; +import java.util.List; +import javax.sql.DataSource; +import org.apache.beam.sdk.PipelineResult; +import org.apache.beam.sdk.io.GenerateSequence; +import org.apache.beam.sdk.io.common.DatabaseTestHelper; +import org.apache.beam.sdk.io.common.HashingFn; +import org.apache.beam.sdk.io.common.TestRow; +import org.apache.beam.sdk.schemas.Schema; +import org.apache.beam.sdk.schemas.transforms.SchemaTransform; +import org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider; +import org.apache.beam.sdk.testing.NeedsRunner; +import org.apache.beam.sdk.testing.PAssert; +import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.transforms.Combine; +import org.apache.beam.sdk.transforms.Count; +import org.apache.beam.sdk.transforms.MapElements; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.transforms.SerializableFunction; +import org.apache.beam.sdk.transforms.Sum; +import org.apache.beam.sdk.transforms.Top; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PCollectionRowTuple; +import org.apache.beam.sdk.values.Row; +import org.apache.beam.sdk.values.TypeDescriptor; +import org.junit.BeforeClass; +import org.junit.Rule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +@RunWith(JUnit4.class) +public class SingleStoreIOSchemaTransformIT { + + private static final String DATABASE_NAME = "SingleStoreIOIT"; + + private static int numberOfRows; + + private static String tableName; + + private static String serverName; + + private static String username; + + private static String password; + + private static Integer port; + + private static SingleStoreIO.DataSourceConfiguration dataSourceConfiguration; + + @BeforeClass + public static void setup() { + SingleStoreIOTestPipelineOptions options; + try { + options = readIOTestPipelineOptions(SingleStoreIOTestPipelineOptions.class); + } catch (IllegalArgumentException e) { + options = null; + } + org.junit.Assume.assumeNotNull(options); + + numberOfRows = options.getNumberOfRecords(); + serverName = options.getSingleStoreServerName(); + username = options.getSingleStoreUsername(); + password = options.getSingleStorePassword(); + port = options.getSingleStorePort(); + tableName = DatabaseTestHelper.getTestTableName("IT"); + dataSourceConfiguration = + SingleStoreIO.DataSourceConfiguration.create(serverName + ":" + port) + .withDatabase(DATABASE_NAME) + .withPassword(password) + .withUsername(username); + } + + @Test + @Category(NeedsRunner.class) + public void testWriteThenRead() throws Exception { + TestHelper.createDatabaseIfNotExists(serverName, port, username, password, DATABASE_NAME); + DataSource dataSource = + new SingleStoreDataSource( + String.format( + "jdbc:singlestore://%s:%d/%s?user=%s&password=%s&allowLocalInfile=TRUE", + serverName, port, DATABASE_NAME, username, password)); + DatabaseTestHelper.createTable(dataSource, tableName); + try { + PipelineResult writeResult = runWrite(); + assertEquals(PipelineResult.State.DONE, writeResult.waitUntilFinish()); + PipelineResult readResult = runRead(); + assertEquals(PipelineResult.State.DONE, readResult.waitUntilFinish()); + PipelineResult readResultWithPartitions = runReadWithPartitions(); + assertEquals(PipelineResult.State.DONE, readResultWithPartitions.waitUntilFinish()); + } finally { + DatabaseTestHelper.deleteTable(dataSource, tableName); + } + } + + @Rule public TestPipeline pipelineWrite = TestPipeline.create(); + @Rule public TestPipeline pipelineRead = TestPipeline.create(); + @Rule public TestPipeline pipelineReadWithPartitions = TestPipeline.create(); + + private PipelineResult runWrite() { + SchemaTransformProvider provider = new SingleStoreSchemaTransformWriteProvider(); + + SingleStoreSchemaTransformWriteConfiguration configuration = + SingleStoreSchemaTransformWriteConfiguration.builder() + .setDataSourceConfiguration(dataSourceConfiguration) + .setTable(tableName) + .setBatchSize(100) + .build(); + + Row configurationRow = configuration.toBeamRow(); + SchemaTransform schemaTransform = provider.from(configurationRow); + PTransform pCollectionRowTupleTransform = + schemaTransform.buildTransform(); + + Schema.Builder schemaBuilder = new Schema.Builder(); + schemaBuilder.addField("id", Schema.FieldType.INT32); + schemaBuilder.addField("name", Schema.FieldType.STRING); + Schema schema = schemaBuilder.build(); + + PCollection rows = + pipelineWrite + .apply(GenerateSequence.from(0).to(numberOfRows)) + .apply(ParDo.of(new TestRow.DeterministicallyConstructTestRowFn())) + .apply( + "Convert TestRows to Rows", + MapElements.into(TypeDescriptor.of(Row.class)) + .via( + (SerializableFunction) + testRow -> { + Row.Builder rowBuilder = Row.withSchema(schema); + rowBuilder.addValue(testRow.id()); + rowBuilder.addValue(testRow.name()); + return rowBuilder.build(); + })) + .setRowSchema(schema); + + PCollectionRowTuple input = + PCollectionRowTuple.of(SingleStoreSchemaTransformWriteProvider.INPUT_TAG, rows); + String tag = provider.outputCollectionNames().get(0); + PCollectionRowTuple output = input.apply(pCollectionRowTupleTransform); + assertTrue(output.has(tag)); + PCollection writtenRows = + output + .get(tag) + .apply( + "Convert Rows to Integers", + MapElements.into(TypeDescriptor.of(Integer.class)) + .via((SerializableFunction) row -> row.getInt32(0))); + + PAssert.thatSingleton(writtenRows.apply("Sum All", Sum.integersGlobally())) + .isEqualTo(numberOfRows); + + return pipelineWrite.run(); + } + + private PipelineResult runRead() { + SchemaTransformProvider provider = new SingleStoreSchemaTransformReadProvider(); + + SingleStoreSchemaTransformReadConfiguration configuration = + SingleStoreSchemaTransformReadConfiguration.builder() + .setDataSourceConfiguration(dataSourceConfiguration) + .setTable(tableName) + .setOutputParallelization(true) + .build(); + + Row configurationRow = configuration.toBeamRow(); + SchemaTransform schemaTransform = provider.from(configurationRow); + PTransform pCollectionRowTupleTransform = + schemaTransform.buildTransform(); + + PCollectionRowTuple input = PCollectionRowTuple.empty(pipelineRead); + String tag = provider.outputCollectionNames().get(0); + PCollectionRowTuple output = input.apply(pCollectionRowTupleTransform); + assertTrue(output.has(tag)); + PCollection namesAndIds = + output + .get(tag) + .apply( + MapElements.into(TypeDescriptor.of(TestRow.class)) + .via( + (SerializableFunction) + row -> TestRow.create(row.getInt32(0), row.getString(1)))); + + testReadResult(namesAndIds); + + return pipelineRead.run(); + } + + private PipelineResult runReadWithPartitions() { + SchemaTransformProvider provider = new SingleStoreSchemaTransformReadWithPartitionsProvider(); + + SingleStoreSchemaTransformReadWithPartitionsConfiguration configuration = + SingleStoreSchemaTransformReadWithPartitionsConfiguration.builder() + .setDataSourceConfiguration(dataSourceConfiguration) + .setTable(tableName) + .setInitialNumReaders(1) + .build(); + + Row configurationRow = configuration.toBeamRow(); + SchemaTransform schemaTransform = provider.from(configurationRow); + PTransform pCollectionRowTupleTransform = + schemaTransform.buildTransform(); + + PCollectionRowTuple input = PCollectionRowTuple.empty(pipelineReadWithPartitions); + String tag = provider.outputCollectionNames().get(0); + PCollectionRowTuple output = input.apply(pCollectionRowTupleTransform); + assertTrue(output.has(tag)); + PCollection namesAndIds = + output + .get(tag) + .apply( + MapElements.into(TypeDescriptor.of(TestRow.class)) + .via( + (SerializableFunction) + row -> TestRow.create(row.getInt32(0), row.getString(1)))); + + testReadResult(namesAndIds); + + return pipelineReadWithPartitions.run(); + } + + private void testReadResult(PCollection namesAndIds) { + PAssert.thatSingleton(namesAndIds.apply("Count All", Count.globally())) + .isEqualTo((long) numberOfRows); + + PCollection consolidatedHashcode = + namesAndIds + .apply(ParDo.of(new TestRow.SelectNameFn())) + .apply("Hash row contents", Combine.globally(new HashingFn()).withoutDefaults()); + PAssert.that(consolidatedHashcode) + .containsInAnyOrder(TestRow.getExpectedHashForRowCount(numberOfRows)); + + PCollection> frontOfList = namesAndIds.apply(Top.smallest(500)); + Iterable expectedFrontOfList = TestRow.getExpectedValues(0, 500); + PAssert.thatSingletonIterable(frontOfList).containsInAnyOrder(expectedFrontOfList); + + PCollection> backOfList = namesAndIds.apply(Top.largest(500)); + Iterable expectedBackOfList = + TestRow.getExpectedValues(numberOfRows - 500, numberOfRows); + PAssert.thatSingletonIterable(backOfList).containsInAnyOrder(expectedBackOfList); + } +} diff --git a/sdks/java/io/singlestore/src/test/java/org/apache/beam/sdk/io/singlestore/SingleStoreUtilTest.java b/sdks/java/io/singlestore/src/test/java/org/apache/beam/sdk/io/singlestore/SingleStoreUtilTest.java index 4b743e43cd1b..2ea7c1095cc3 100644 --- a/sdks/java/io/singlestore/src/test/java/org/apache/beam/sdk/io/singlestore/SingleStoreUtilTest.java +++ b/sdks/java/io/singlestore/src/test/java/org/apache/beam/sdk/io/singlestore/SingleStoreUtilTest.java @@ -79,6 +79,15 @@ public TestRow mapRow(ResultSet resultSet) throws Exception { } } + @Test + public void testInferCoderFromCoder() { + SchemaRegistry sr = SchemaRegistry.createDefault(); + CoderRegistry cr = CoderRegistry.createDefault(); + Coder c = SerializableCoder.of(TestRow.class); + + assertEquals(c, SingleStoreUtil.inferCoder(c, new TestRowMapper(), cr, sr, LOG)); + } + @Test public void testInferCoderFromSchemaRegistry() { SchemaRegistry sr = SchemaRegistry.createDefault(); @@ -86,7 +95,7 @@ public void testInferCoderFromSchemaRegistry() { Coder c = SerializableCoder.of(TestRow.class); cr.registerCoderForClass(TestRow.class, c); - assertEquals(c, SingleStoreUtil.inferCoder(new TestRowMapper(), cr, sr, LOG)); + assertEquals(c, SingleStoreUtil.inferCoder(null, new TestRowMapper(), cr, sr, LOG)); } @Test @@ -96,7 +105,7 @@ public void testInferCoderFromCoderRegistry() throws NoSuchSchemaException { sr.registerPOJO(TestRow.class); Coder c = sr.getSchemaCoder(TestRow.class); - assertEquals(c, SingleStoreUtil.inferCoder(new TestRowMapper(), cr, sr, LOG)); + assertEquals(c, SingleStoreUtil.inferCoder(null, new TestRowMapper(), cr, sr, LOG)); } @Test diff --git a/sdks/java/io/singlestore/src/test/java/org/apache/beam/sdk/io/singlestore/TestHelper.java b/sdks/java/io/singlestore/src/test/java/org/apache/beam/sdk/io/singlestore/TestHelper.java index 7c0acc539098..e04785db4ee3 100644 --- a/sdks/java/io/singlestore/src/test/java/org/apache/beam/sdk/io/singlestore/TestHelper.java +++ b/sdks/java/io/singlestore/src/test/java/org/apache/beam/sdk/io/singlestore/TestHelper.java @@ -17,9 +17,14 @@ */ package org.apache.beam.sdk.io.singlestore; +import com.singlestore.jdbc.SingleStoreDataSource; +import java.sql.Connection; import java.sql.ResultSet; +import java.sql.SQLException; +import java.sql.Statement; import java.util.ArrayList; import java.util.List; +import javax.sql.DataSource; import org.apache.beam.sdk.io.common.TestRow; public class TestHelper { @@ -67,4 +72,18 @@ String getPassword() { return "secretPass"; } } + + public static void createDatabaseIfNotExists( + String serverName, Integer port, String username, String password, String databaseName) + throws SQLException { + DataSource dataSource = + new SingleStoreDataSource( + String.format( + "jdbc:singlestore://%s:%d/?user=%s&password=%s&allowLocalInfile=TRUE", + serverName, port, username, password)); + try (Connection conn = dataSource.getConnection(); + Statement stmt = conn.createStatement()) { + stmt.executeQuery(String.format("CREATE DATABASE IF NOT EXISTS %s", databaseName)); + } + } } From 69a1d21199fa1c0195a6b652ec1da2d3a969db5b Mon Sep 17 00:00:00 2001 From: Adalbert Makarovych Date: Thu, 24 Nov 2022 10:34:13 +0200 Subject: [PATCH 02/21] Fixed nullable errors --- .../SingleStoreDefaultRowMapper.java | 16 ++++++++++++-- .../SingleStoreDefaultUserDataMapper.java | 7 ++++-- .../sdk/io/singlestore/SingleStoreIO.java | 22 ++++++++++++------- ...ingleStoreSchemaTransformReadProvider.java | 21 +++++++++++------- ...maTransformReadWithPartitionsProvider.java | 21 +++++++++++------- ...ngleStoreSchemaTransformWriteProvider.java | 16 +++++++++----- 6 files changed, 69 insertions(+), 34 deletions(-) diff --git a/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreDefaultRowMapper.java b/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreDefaultRowMapper.java index e10da5acaf40..a09d146ab088 100644 --- a/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreDefaultRowMapper.java +++ b/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreDefaultRowMapper.java @@ -59,6 +59,7 @@ import org.apache.beam.sdk.schemas.logicaltypes.VariableBytes; import org.apache.beam.sdk.schemas.logicaltypes.VariableString; import org.apache.beam.sdk.values.Row; +import org.checkerframework.checker.nullness.qual.Nullable; import org.joda.time.DateTime; import org.joda.time.LocalDate; import org.joda.time.chrono.ISOChronology; @@ -92,7 +93,7 @@ public Row mapRow(ResultSet resultSet) throws Exception { for (int i = 0; i < schema.getFieldCount(); i++) { Object value = converters.get(i).getValue(resultSet, i + 1); - if (resultSet.wasNull()) { + if (resultSet.wasNull() || value == null) { rowBuilder.addValue(null); } else { rowBuilder.addValue(value); @@ -103,7 +104,7 @@ public Row mapRow(ResultSet resultSet) throws Exception { } abstract static class ResultSetFieldConverter implements Serializable { - abstract Object getValue(ResultSet rs, Integer index) throws SQLException; + abstract @Nullable Object getValue(ResultSet rs, Integer index) throws SQLException; Schema.Field getSchemaField(ResultSetMetaData md, Integer index) throws SQLException { String label = md.getColumnLabel(index); @@ -120,6 +121,7 @@ abstract Schema.FieldType getSchemaFieldType(ResultSetMetaData md, Integer index */ @FunctionalInterface interface ResultSetFieldExtractor extends Serializable { + @Nullable Object extract(ResultSet rs, Integer index) throws SQLException; } @@ -176,6 +178,7 @@ public DirectResultSetFieldConverter( } @Override + @Nullable Object getValue(ResultSet rs, Integer index) throws SQLException { return extractor.extract(rs, index); } @@ -188,6 +191,7 @@ Schema.FieldType getSchemaFieldType(ResultSetMetaData md, Integer index) { static class CharResultSetFieldConverter extends ResultSetFieldConverter { @Override + @Nullable Object getValue(ResultSet rs, Integer index) throws SQLException { return rs.getString(index); } @@ -201,6 +205,7 @@ Schema.FieldType getSchemaFieldType(ResultSetMetaData md, Integer index) throws static class BinaryResultSetFieldConverter extends ResultSetFieldConverter { @Override + @Nullable Object getValue(ResultSet rs, Integer index) throws SQLException { return rs.getBytes(index); } @@ -214,6 +219,7 @@ Schema.FieldType getSchemaFieldType(ResultSetMetaData md, Integer index) throws static class TimestampResultSetFieldConverter extends ResultSetFieldConverter { @Override + @Nullable Object getValue(ResultSet rs, Integer index) throws SQLException { Timestamp ts = rs.getTimestamp(index, Calendar.getInstance(TimeZone.getTimeZone(ZoneOffset.UTC))); @@ -231,6 +237,7 @@ Schema.FieldType getSchemaFieldType(ResultSetMetaData md, Integer index) { static class TimeResultSetFieldConverter extends ResultSetFieldConverter { @Override + @Nullable Object getValue(ResultSet rs, Integer index) throws SQLException { Time time = rs.getTime(index, Calendar.getInstance(TimeZone.getTimeZone(ZoneOffset.UTC))); if (time == null) { @@ -248,6 +255,7 @@ Schema.FieldType getSchemaFieldType(ResultSetMetaData md, Integer index) { static class DateResultSetFieldConverter extends ResultSetFieldConverter { @Override + @Nullable Object getValue(ResultSet rs, Integer index) throws SQLException { // TODO(https://github.com/apache/beam/issues/19215) import when joda LocalDate is removed. java.time.LocalDate date = rs.getObject(index, java.time.LocalDate.class); @@ -268,5 +276,9 @@ public static class SingleStoreDefaultRowMapperCreationException extends Runtime SingleStoreDefaultRowMapperCreationException(String message, Throwable cause) { super(message, cause); } + + SingleStoreDefaultRowMapperCreationException(String message) { + super(message); + } } } diff --git a/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreDefaultUserDataMapper.java b/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreDefaultUserDataMapper.java index e7d4760d2bbc..2bb2b17f4b13 100644 --- a/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreDefaultUserDataMapper.java +++ b/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreDefaultUserDataMapper.java @@ -23,13 +23,14 @@ import java.util.List; import org.apache.beam.sdk.schemas.Schema; import org.apache.beam.sdk.values.Row; +import org.checkerframework.checker.nullness.qual.Nullable; import org.joda.time.Instant; import org.joda.time.format.DateTimeFormat; import org.joda.time.format.DateTimeFormatter; final class SingleStoreDefaultUserDataMapper implements SingleStoreIO.UserDataMapper { - private DateTimeFormatter formatter = null; + private @Nullable DateTimeFormatter formatter = null; private DateTimeFormatter getFormatter() { if (formatter == null) { @@ -41,7 +42,9 @@ private DateTimeFormatter getFormatter() { private String convertLogicalTypeFieldToString(Schema.FieldType type, Object value) { Schema.LogicalType logicalType = type.getLogicalType(); - assert logicalType != null; + if (logicalType == null) { + throw new UnsupportedOperationException("Failed to extract logical type"); + } Schema.FieldType baseType = logicalType.getBaseType(); Object baseValue = logicalType.toBaseType(value); diff --git a/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreIO.java b/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreIO.java index 22668901f56d..0ca93193bee6 100644 --- a/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreIO.java +++ b/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreIO.java @@ -327,8 +327,7 @@ public DataSource getDataSource() { String database = SingleStoreUtil.getArgumentWithDefault(getDatabase(), ""); String connectionProperties = SingleStoreUtil.getArgumentWithDefault(getConnectionProperties(), ""); - connectionProperties += - (connectionProperties.isEmpty() ? "" : ";") + "allowLocalInfile=TRUE"; + connectionProperties += (connectionProperties.isEmpty() ? "" : ";") + "allowLocalInfile=TRUE"; String username = getUsername(); String password = getPassword(); @@ -573,6 +572,10 @@ private static SingleStoreDefaultRowMapper getRowMapper( conn.prepareStatement(String.format("SELECT * FROM (%s) LIMIT 0", query)); try { ResultSetMetaData md = stmt.getMetaData(); + if (md == null) { + throw new SingleStoreDefaultRowMapper.SingleStoreDefaultRowMapperCreationException( + "Failed to retrieve statement metadata"); + } return new SingleStoreDefaultRowMapper(md); } finally { stmt.close(); @@ -650,6 +653,8 @@ public PCollection expand(PBegin input) { Preconditions.checkArgumentNotNull( dataSourceConfiguration, "withDataSourceConfiguration() is required"); String actualQuery = SingleStoreUtil.getSelectQuery(getTable(), getQuery()); + Boolean outputParallelization = getOutputParallelization(); + StatementPreparator statementPreparator = getStatementPreparator(); SingleStoreDefaultRowMapper rowMapper = getRowMapper(dataSourceConfiguration, actualQuery); @@ -660,12 +665,12 @@ public PCollection expand(PBegin input) { .withRowMapper(rowMapper) .withCoder(RowCoder.of(rowMapper.getSchema())); - if (getOutputParallelization() != null) { - read = read.withOutputParallelization(getOutputParallelization()); + if (outputParallelization != null) { + read = read.withOutputParallelization(outputParallelization); } - if (getStatementPreparator() != null) { - read = read.withStatementPreparator(getStatementPreparator()); + if (statementPreparator != null) { + read = read.withStatementPreparator(statementPreparator); } PCollection output = input.apply(read); @@ -937,6 +942,7 @@ public PCollection expand(PBegin input) { Preconditions.checkArgumentNotNull( dataSourceConfiguration, "withDataSourceConfiguration() is required"); String actualQuery = SingleStoreUtil.getSelectQuery(getTable(), getQuery()); + Integer initialNumReaders = getInitialNumReaders(); SingleStoreDefaultRowMapper rowMapper = getRowMapper(dataSourceConfiguration, actualQuery); @@ -947,8 +953,8 @@ public PCollection expand(PBegin input) { .withRowMapper(rowMapper) .withCoder(RowCoder.of(rowMapper.getSchema())); - if (getInitialNumReaders() != null) { - readWithPartitions = readWithPartitions.withInitialNumReaders(getInitialNumReaders()); + if (initialNumReaders != null) { + readWithPartitions = readWithPartitions.withInitialNumReaders(initialNumReaders); } PCollection output = input.apply(readWithPartitions); diff --git a/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreSchemaTransformReadProvider.java b/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreSchemaTransformReadProvider.java index b008beb3b1b0..2ed8e5f0574b 100644 --- a/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreSchemaTransformReadProvider.java +++ b/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreSchemaTransformReadProvider.java @@ -112,23 +112,28 @@ public PCollectionRowTuple expand(PCollectionRowTuple input) { "%s %s input is expected to be empty", input.getClass().getSimpleName(), getClass().getSimpleName())); } + SingleStoreIO.DataSourceConfiguration dataSourceConfiguration = + configuration.getDataSourceConfiguration(); + String table = configuration.getTable(); + String query = configuration.getQuery(); + Boolean outputParallelization = configuration.getOutputParallelization(); SingleStoreIO.ReadRows read = SingleStoreIO.readRows(); - if (configuration.getDataSourceConfiguration() != null) { - read = read.withDataSourceConfiguration(configuration.getDataSourceConfiguration()); + if (dataSourceConfiguration != null) { + read = read.withDataSourceConfiguration(dataSourceConfiguration); } - if (configuration.getTable() != null) { - read = read.withTable(configuration.getTable()); + if (table != null) { + read = read.withTable(table); } - if (configuration.getQuery() != null) { - read = read.withQuery(configuration.getQuery()); + if (query != null) { + read = read.withQuery(query); } - if (configuration.getOutputParallelization() != null) { - read = read.withOutputParallelization(configuration.getOutputParallelization()); + if (outputParallelization != null) { + read = read.withOutputParallelization(outputParallelization); } PCollection rows = input.getPipeline().apply(read); diff --git a/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreSchemaTransformReadWithPartitionsProvider.java b/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreSchemaTransformReadWithPartitionsProvider.java index 8d608165c9c3..eb2b59bdf0ea 100644 --- a/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreSchemaTransformReadWithPartitionsProvider.java +++ b/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreSchemaTransformReadWithPartitionsProvider.java @@ -116,23 +116,28 @@ public PCollectionRowTuple expand(PCollectionRowTuple input) { "%s %s input is expected to be empty", input.getClass().getSimpleName(), getClass().getSimpleName())); } + SingleStoreIO.DataSourceConfiguration dataSourceConfiguration = + configuration.getDataSourceConfiguration(); + String table = configuration.getTable(); + String query = configuration.getQuery(); + Integer initialNumReaders = configuration.getInitialNumReaders(); SingleStoreIO.ReadWithPartitionsRows read = SingleStoreIO.readWithPartitionsRows(); - if (configuration.getDataSourceConfiguration() != null) { - read = read.withDataSourceConfiguration(configuration.getDataSourceConfiguration()); + if (dataSourceConfiguration != null) { + read = read.withDataSourceConfiguration(dataSourceConfiguration); } - if (configuration.getTable() != null) { - read = read.withTable(configuration.getTable()); + if (table != null) { + read = read.withTable(table); } - if (configuration.getQuery() != null) { - read = read.withQuery(configuration.getQuery()); + if (query != null) { + read = read.withQuery(query); } - if (configuration.getInitialNumReaders() != null) { - read = read.withInitialNumReaders(configuration.getInitialNumReaders()); + if (initialNumReaders != null) { + read = read.withInitialNumReaders(initialNumReaders); } PCollection rows = input.getPipeline().apply(read); diff --git a/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreSchemaTransformWriteProvider.java b/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreSchemaTransformWriteProvider.java index 49ffa01cd2e8..165ab3fac179 100644 --- a/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreSchemaTransformWriteProvider.java +++ b/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreSchemaTransformWriteProvider.java @@ -116,19 +116,23 @@ public PCollectionRowTuple expand(PCollectionRowTuple input) { "%s %s is missing expected tag: %s", getClass().getSimpleName(), input.getClass().getSimpleName(), INPUT_TAG)); } + SingleStoreIO.DataSourceConfiguration dataSourceConfiguration = + configuration.getDataSourceConfiguration(); + String table = configuration.getTable(); + Integer batchSize = configuration.getBatchSize(); SingleStoreIO.Write write = SingleStoreIO.writeRows(); - if (configuration.getDataSourceConfiguration() != null) { - write = write.withDataSourceConfiguration(configuration.getDataSourceConfiguration()); + if (dataSourceConfiguration != null) { + write = write.withDataSourceConfiguration(dataSourceConfiguration); } - if (configuration.getTable() != null) { - write = write.withTable(configuration.getTable()); + if (table != null) { + write = write.withTable(table); } - if (configuration.getBatchSize() != null) { - write = write.withBatchSize(configuration.getBatchSize()); + if (batchSize != null) { + write = write.withBatchSize(batchSize); } PCollection res = input.get(INPUT_TAG).apply(write); From 08b31320d8698c5592ecb77d36628beea186fc84 Mon Sep 17 00:00:00 2001 From: Adalbert Makarovych Date: Thu, 24 Nov 2022 10:50:00 +0200 Subject: [PATCH 03/21] Changed to don't use .* form of import --- .../SingleStoreDefaultRowMapperTest.java | 19 ++++++++++++++++++- 1 file changed, 18 insertions(+), 1 deletion(-) diff --git a/sdks/java/io/singlestore/src/test/java/org/apache/beam/sdk/io/singlestore/SingleStoreDefaultRowMapperTest.java b/sdks/java/io/singlestore/src/test/java/org/apache/beam/sdk/io/singlestore/SingleStoreDefaultRowMapperTest.java index bf5f8f2113e3..b91a78dfc639 100644 --- a/sdks/java/io/singlestore/src/test/java/org/apache/beam/sdk/io/singlestore/SingleStoreDefaultRowMapperTest.java +++ b/sdks/java/io/singlestore/src/test/java/org/apache/beam/sdk/io/singlestore/SingleStoreDefaultRowMapperTest.java @@ -17,7 +17,24 @@ */ package org.apache.beam.sdk.io.singlestore; -import static java.sql.Types.*; +import static java.sql.Types.BIGINT; +import static java.sql.Types.BINARY; +import static java.sql.Types.BIT; +import static java.sql.Types.CHAR; +import static java.sql.Types.DECIMAL; +import static java.sql.Types.DOUBLE; +import static java.sql.Types.INTEGER; +import static java.sql.Types.LONGVARBINARY; +import static java.sql.Types.LONGVARCHAR; +import static java.sql.Types.NULL; +import static java.sql.Types.REAL; +import static java.sql.Types.SMALLINT; +import static java.sql.Types.TIME; +import static java.sql.Types.TIMESTAMP; +import static java.sql.Types.TINYINT; +import static java.sql.Types.VARBINARY; +import static java.sql.Types.VARCHAR; +import static java.util.Calendar.DATE; import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNull; From caf46da1b493e79b3b1dc98dbabd6e6b2a99de0e Mon Sep 17 00:00:00 2001 From: Adalbert Makarovych Date: Fri, 25 Nov 2022 12:50:22 +0200 Subject: [PATCH 04/21] Changed formatter field to be transient --- .../SingleStoreDefaultUserDataMapper.java | 13 ++----------- 1 file changed, 2 insertions(+), 11 deletions(-) diff --git a/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreDefaultUserDataMapper.java b/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreDefaultUserDataMapper.java index 2bb2b17f4b13..6c1def1c1cca 100644 --- a/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreDefaultUserDataMapper.java +++ b/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreDefaultUserDataMapper.java @@ -23,22 +23,13 @@ import java.util.List; import org.apache.beam.sdk.schemas.Schema; import org.apache.beam.sdk.values.Row; -import org.checkerframework.checker.nullness.qual.Nullable; import org.joda.time.Instant; import org.joda.time.format.DateTimeFormat; import org.joda.time.format.DateTimeFormatter; final class SingleStoreDefaultUserDataMapper implements SingleStoreIO.UserDataMapper { - private @Nullable DateTimeFormatter formatter = null; - - private DateTimeFormatter getFormatter() { - if (formatter == null) { - formatter = DateTimeFormat.forPattern("yyyy-MM-DD' 'HH:mm:ss.SSS"); - } - - return formatter; - } + private final transient DateTimeFormatter formatter = DateTimeFormat.forPattern("yyyy-MM-DD' 'HH:mm:ss.SSS"); private String convertLogicalTypeFieldToString(Schema.FieldType type, Object value) { Schema.LogicalType logicalType = type.getLogicalType(); @@ -70,7 +61,7 @@ private String convertFieldToString(Schema.FieldType type, Object value) { case STRING: return (String) value; case DATETIME: - return getFormatter().print((Instant) value); + return formatter.print((Instant) value); case BOOLEAN: return ((Boolean) value) ? "1" : "0"; case BYTES: From 302ceac0e18a1a99b27063b24dcbe4eac222a7ed Mon Sep 17 00:00:00 2001 From: Adalbert Makarovych Date: Fri, 25 Nov 2022 13:01:03 +0200 Subject: [PATCH 05/21] Nit reformatting --- .../sdk/io/singlestore/SingleStoreDefaultUserDataMapper.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreDefaultUserDataMapper.java b/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreDefaultUserDataMapper.java index 6c1def1c1cca..fd3fb8c4c380 100644 --- a/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreDefaultUserDataMapper.java +++ b/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreDefaultUserDataMapper.java @@ -29,7 +29,8 @@ final class SingleStoreDefaultUserDataMapper implements SingleStoreIO.UserDataMapper { - private final transient DateTimeFormatter formatter = DateTimeFormat.forPattern("yyyy-MM-DD' 'HH:mm:ss.SSS"); + private final transient DateTimeFormatter formatter = + DateTimeFormat.forPattern("yyyy-MM-DD' 'HH:mm:ss.SSS"); private String convertLogicalTypeFieldToString(Schema.FieldType type, Object value) { Schema.LogicalType logicalType = type.getLogicalType(); From b5e954357b07d2624dc733c713431e75153baec9 Mon Sep 17 00:00:00 2001 From: Adalbert Makarovych Date: Fri, 25 Nov 2022 13:40:58 +0200 Subject: [PATCH 06/21] Fixed bugs in tests --- .../io/singlestore/SingleStoreDefaultRowMapperTest.java | 2 +- .../singlestore/SingleStoreDefaultUserDataMapperTest.java | 8 ++++---- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/sdks/java/io/singlestore/src/test/java/org/apache/beam/sdk/io/singlestore/SingleStoreDefaultRowMapperTest.java b/sdks/java/io/singlestore/src/test/java/org/apache/beam/sdk/io/singlestore/SingleStoreDefaultRowMapperTest.java index b91a78dfc639..87afb6069ee0 100644 --- a/sdks/java/io/singlestore/src/test/java/org/apache/beam/sdk/io/singlestore/SingleStoreDefaultRowMapperTest.java +++ b/sdks/java/io/singlestore/src/test/java/org/apache/beam/sdk/io/singlestore/SingleStoreDefaultRowMapperTest.java @@ -21,6 +21,7 @@ import static java.sql.Types.BINARY; import static java.sql.Types.BIT; import static java.sql.Types.CHAR; +import static java.sql.Types.DATE; import static java.sql.Types.DECIMAL; import static java.sql.Types.DOUBLE; import static java.sql.Types.INTEGER; @@ -34,7 +35,6 @@ import static java.sql.Types.TINYINT; import static java.sql.Types.VARBINARY; import static java.sql.Types.VARCHAR; -import static java.util.Calendar.DATE; import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNull; diff --git a/sdks/java/io/singlestore/src/test/java/org/apache/beam/sdk/io/singlestore/SingleStoreDefaultUserDataMapperTest.java b/sdks/java/io/singlestore/src/test/java/org/apache/beam/sdk/io/singlestore/SingleStoreDefaultUserDataMapperTest.java index a6b52e2f4d4f..6472ad13cff6 100644 --- a/sdks/java/io/singlestore/src/test/java/org/apache/beam/sdk/io/singlestore/SingleStoreDefaultUserDataMapperTest.java +++ b/sdks/java/io/singlestore/src/test/java/org/apache/beam/sdk/io/singlestore/SingleStoreDefaultUserDataMapperTest.java @@ -66,10 +66,10 @@ public void testBigNegativeNumbers() { assertEquals(7, res.size()); assertEquals("-128", res.get(0)); assertEquals("-32768", res.get(1)); - assertEquals("10", res.get(2)); - assertEquals("10", res.get(3)); - assertEquals("10.1", res.get(4)); - assertEquals("10.1", res.get(5)); + assertEquals("-2147483648", res.get(2)); + assertEquals("-9223372036854775808", res.get(3)); + assertEquals("-3.4028235E38", res.get(4)); + assertEquals("-1.7976931348623157E308", res.get(5)); assertEquals("-10000000000000.1000000000000000000000", res.get(6)); } From 4e2e7bc1d6ee6ad48f63d77498aaddf2060cca05 Mon Sep 17 00:00:00 2001 From: Adalbert Makarovych Date: Mon, 28 Nov 2022 18:42:39 +0200 Subject: [PATCH 07/21] Moved schema transform classes to the separate folder --- .../SingleStoreSchemaTransformReadConfiguration.java | 7 +++++-- .../SingleStoreSchemaTransformReadProvider.java | 3 ++- ...toreSchemaTransformReadWithPartitionsConfiguration.java | 7 +++++-- ...ngleStoreSchemaTransformReadWithPartitionsProvider.java | 3 ++- .../SingleStoreSchemaTransformWriteConfiguration.java | 7 +++++-- .../SingleStoreSchemaTransformWriteProvider.java | 5 +++-- .../sdk/io/singlestore/SingleStoreIOSchemaTransformIT.java | 6 ++++++ 7 files changed, 28 insertions(+), 10 deletions(-) rename sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/{ => schematransform}/SingleStoreSchemaTransformReadConfiguration.java (91%) rename sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/{ => schematransform}/SingleStoreSchemaTransformReadProvider.java (97%) rename sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/{ => schematransform}/SingleStoreSchemaTransformReadWithPartitionsConfiguration.java (91%) rename sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/{ => schematransform}/SingleStoreSchemaTransformReadWithPartitionsProvider.java (97%) rename sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/{ => schematransform}/SingleStoreSchemaTransformWriteConfiguration.java (91%) rename sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/{ => schematransform}/SingleStoreSchemaTransformWriteProvider.java (97%) diff --git a/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreSchemaTransformReadConfiguration.java b/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/schematransform/SingleStoreSchemaTransformReadConfiguration.java similarity index 91% rename from sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreSchemaTransformReadConfiguration.java rename to sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/schematransform/SingleStoreSchemaTransformReadConfiguration.java index 74ed7edd2736..c3100b12587c 100644 --- a/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreSchemaTransformReadConfiguration.java +++ b/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/schematransform/SingleStoreSchemaTransformReadConfiguration.java @@ -15,10 +15,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.sdk.io.singlestore; +package org.apache.beam.sdk.io.singlestore.schematransform; import com.google.auto.value.AutoValue; import javax.annotation.Nullable; + +import org.apache.beam.sdk.io.singlestore.SingleStoreIO; +import org.apache.beam.sdk.io.singlestore.schematransform.AutoValue_SingleStoreSchemaTransformReadConfiguration; import org.apache.beam.sdk.schemas.AutoValueSchema; import org.apache.beam.sdk.schemas.annotations.DefaultSchema; import org.apache.beam.sdk.transforms.SerializableFunction; @@ -46,7 +49,7 @@ public static Builder builder() { ROW_SERIALIZABLE_FUNCTION = AUTO_VALUE_SCHEMA.toRowFunction(TYPE_DESCRIPTOR); /** Serializes configuration to a {@link Row}. */ - Row toBeamRow() { + public Row toBeamRow() { return ROW_SERIALIZABLE_FUNCTION.apply(this); } diff --git a/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreSchemaTransformReadProvider.java b/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/schematransform/SingleStoreSchemaTransformReadProvider.java similarity index 97% rename from sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreSchemaTransformReadProvider.java rename to sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/schematransform/SingleStoreSchemaTransformReadProvider.java index 2ed8e5f0574b..32eb114d2a55 100644 --- a/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreSchemaTransformReadProvider.java +++ b/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/schematransform/SingleStoreSchemaTransformReadProvider.java @@ -15,10 +15,11 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.sdk.io.singlestore; +package org.apache.beam.sdk.io.singlestore.schematransform; import java.util.Collections; import java.util.List; +import org.apache.beam.sdk.io.singlestore.SingleStoreIO; import org.apache.beam.sdk.schemas.Schema; import org.apache.beam.sdk.schemas.transforms.SchemaTransform; import org.apache.beam.sdk.schemas.transforms.TypedSchemaTransformProvider; diff --git a/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreSchemaTransformReadWithPartitionsConfiguration.java b/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/schematransform/SingleStoreSchemaTransformReadWithPartitionsConfiguration.java similarity index 91% rename from sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreSchemaTransformReadWithPartitionsConfiguration.java rename to sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/schematransform/SingleStoreSchemaTransformReadWithPartitionsConfiguration.java index 2e0a89a3691f..e21c2b0fb319 100644 --- a/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreSchemaTransformReadWithPartitionsConfiguration.java +++ b/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/schematransform/SingleStoreSchemaTransformReadWithPartitionsConfiguration.java @@ -15,10 +15,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.sdk.io.singlestore; +package org.apache.beam.sdk.io.singlestore.schematransform; import com.google.auto.value.AutoValue; import javax.annotation.Nullable; + +import org.apache.beam.sdk.io.singlestore.SingleStoreIO; +import org.apache.beam.sdk.io.singlestore.schematransform.AutoValue_SingleStoreSchemaTransformReadWithPartitionsConfiguration; import org.apache.beam.sdk.schemas.AutoValueSchema; import org.apache.beam.sdk.schemas.annotations.DefaultSchema; import org.apache.beam.sdk.transforms.SerializableFunction; @@ -49,7 +52,7 @@ public static Builder builder() { ROW_SERIALIZABLE_FUNCTION = AUTO_VALUE_SCHEMA.toRowFunction(TYPE_DESCRIPTOR); /** Serializes configuration to a {@link Row}. */ - Row toBeamRow() { + public Row toBeamRow() { return ROW_SERIALIZABLE_FUNCTION.apply(this); } diff --git a/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreSchemaTransformReadWithPartitionsProvider.java b/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/schematransform/SingleStoreSchemaTransformReadWithPartitionsProvider.java similarity index 97% rename from sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreSchemaTransformReadWithPartitionsProvider.java rename to sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/schematransform/SingleStoreSchemaTransformReadWithPartitionsProvider.java index eb2b59bdf0ea..41f3f174c9ce 100644 --- a/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreSchemaTransformReadWithPartitionsProvider.java +++ b/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/schematransform/SingleStoreSchemaTransformReadWithPartitionsProvider.java @@ -15,10 +15,11 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.sdk.io.singlestore; +package org.apache.beam.sdk.io.singlestore.schematransform; import java.util.Collections; import java.util.List; +import org.apache.beam.sdk.io.singlestore.SingleStoreIO; import org.apache.beam.sdk.schemas.Schema; import org.apache.beam.sdk.schemas.transforms.SchemaTransform; import org.apache.beam.sdk.schemas.transforms.TypedSchemaTransformProvider; diff --git a/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreSchemaTransformWriteConfiguration.java b/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/schematransform/SingleStoreSchemaTransformWriteConfiguration.java similarity index 91% rename from sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreSchemaTransformWriteConfiguration.java rename to sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/schematransform/SingleStoreSchemaTransformWriteConfiguration.java index c5694daf8045..b328065de87e 100644 --- a/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreSchemaTransformWriteConfiguration.java +++ b/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/schematransform/SingleStoreSchemaTransformWriteConfiguration.java @@ -15,10 +15,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.sdk.io.singlestore; +package org.apache.beam.sdk.io.singlestore.schematransform; import com.google.auto.value.AutoValue; import javax.annotation.Nullable; + +import org.apache.beam.sdk.io.singlestore.SingleStoreIO; +import org.apache.beam.sdk.io.singlestore.schematransform.AutoValue_SingleStoreSchemaTransformWriteConfiguration; import org.apache.beam.sdk.schemas.AutoValueSchema; import org.apache.beam.sdk.schemas.annotations.DefaultSchema; import org.apache.beam.sdk.transforms.SerializableFunction; @@ -46,7 +49,7 @@ public static Builder builder() { ROW_SERIALIZABLE_FUNCTION = AUTO_VALUE_SCHEMA.toRowFunction(TYPE_DESCRIPTOR); /** Serializes configuration to a {@link Row}. */ - Row toBeamRow() { + public Row toBeamRow() { return ROW_SERIALIZABLE_FUNCTION.apply(this); } diff --git a/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreSchemaTransformWriteProvider.java b/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/schematransform/SingleStoreSchemaTransformWriteProvider.java similarity index 97% rename from sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreSchemaTransformWriteProvider.java rename to sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/schematransform/SingleStoreSchemaTransformWriteProvider.java index 165ab3fac179..0b23fdd0e323 100644 --- a/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreSchemaTransformWriteProvider.java +++ b/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/schematransform/SingleStoreSchemaTransformWriteProvider.java @@ -15,10 +15,11 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.sdk.io.singlestore; +package org.apache.beam.sdk.io.singlestore.schematransform; import java.util.Collections; import java.util.List; +import org.apache.beam.sdk.io.singlestore.SingleStoreIO; import org.apache.beam.sdk.schemas.Schema; import org.apache.beam.sdk.schemas.transforms.SchemaTransform; import org.apache.beam.sdk.schemas.transforms.TypedSchemaTransformProvider; @@ -39,7 +40,7 @@ public class SingleStoreSchemaTransformWriteProvider private static final String API = "singlestore"; private static final String OUTPUT_TAG = "OUTPUT"; - static final String INPUT_TAG = "INPUT"; + public static final String INPUT_TAG = "INPUT"; /** Returns the expected class of the configuration. */ @Override diff --git a/sdks/java/io/singlestore/src/test/java/org/apache/beam/sdk/io/singlestore/SingleStoreIOSchemaTransformIT.java b/sdks/java/io/singlestore/src/test/java/org/apache/beam/sdk/io/singlestore/SingleStoreIOSchemaTransformIT.java index 8542e0631b41..619d3422b953 100644 --- a/sdks/java/io/singlestore/src/test/java/org/apache/beam/sdk/io/singlestore/SingleStoreIOSchemaTransformIT.java +++ b/sdks/java/io/singlestore/src/test/java/org/apache/beam/sdk/io/singlestore/SingleStoreIOSchemaTransformIT.java @@ -29,6 +29,12 @@ import org.apache.beam.sdk.io.common.DatabaseTestHelper; import org.apache.beam.sdk.io.common.HashingFn; import org.apache.beam.sdk.io.common.TestRow; +import org.apache.beam.sdk.io.singlestore.schematransform.SingleStoreSchemaTransformReadConfiguration; +import org.apache.beam.sdk.io.singlestore.schematransform.SingleStoreSchemaTransformReadProvider; +import org.apache.beam.sdk.io.singlestore.schematransform.SingleStoreSchemaTransformReadWithPartitionsConfiguration; +import org.apache.beam.sdk.io.singlestore.schematransform.SingleStoreSchemaTransformReadWithPartitionsProvider; +import org.apache.beam.sdk.io.singlestore.schematransform.SingleStoreSchemaTransformWriteConfiguration; +import org.apache.beam.sdk.io.singlestore.schematransform.SingleStoreSchemaTransformWriteProvider; import org.apache.beam.sdk.schemas.Schema; import org.apache.beam.sdk.schemas.transforms.SchemaTransform; import org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider; From d59828a330c7dcec407d8af70bd2fb0edc05cc81 Mon Sep 17 00:00:00 2001 From: Adalbert Makarovych Date: Mon, 28 Nov 2022 18:52:00 +0200 Subject: [PATCH 08/21] Removed unused imports --- .../SingleStoreSchemaTransformReadConfiguration.java | 2 -- ...ngleStoreSchemaTransformReadWithPartitionsConfiguration.java | 2 -- .../SingleStoreSchemaTransformWriteConfiguration.java | 2 -- 3 files changed, 6 deletions(-) diff --git a/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/schematransform/SingleStoreSchemaTransformReadConfiguration.java b/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/schematransform/SingleStoreSchemaTransformReadConfiguration.java index c3100b12587c..d2d21c35d8fa 100644 --- a/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/schematransform/SingleStoreSchemaTransformReadConfiguration.java +++ b/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/schematransform/SingleStoreSchemaTransformReadConfiguration.java @@ -19,9 +19,7 @@ import com.google.auto.value.AutoValue; import javax.annotation.Nullable; - import org.apache.beam.sdk.io.singlestore.SingleStoreIO; -import org.apache.beam.sdk.io.singlestore.schematransform.AutoValue_SingleStoreSchemaTransformReadConfiguration; import org.apache.beam.sdk.schemas.AutoValueSchema; import org.apache.beam.sdk.schemas.annotations.DefaultSchema; import org.apache.beam.sdk.transforms.SerializableFunction; diff --git a/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/schematransform/SingleStoreSchemaTransformReadWithPartitionsConfiguration.java b/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/schematransform/SingleStoreSchemaTransformReadWithPartitionsConfiguration.java index e21c2b0fb319..67f0ab8f5b9a 100644 --- a/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/schematransform/SingleStoreSchemaTransformReadWithPartitionsConfiguration.java +++ b/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/schematransform/SingleStoreSchemaTransformReadWithPartitionsConfiguration.java @@ -19,9 +19,7 @@ import com.google.auto.value.AutoValue; import javax.annotation.Nullable; - import org.apache.beam.sdk.io.singlestore.SingleStoreIO; -import org.apache.beam.sdk.io.singlestore.schematransform.AutoValue_SingleStoreSchemaTransformReadWithPartitionsConfiguration; import org.apache.beam.sdk.schemas.AutoValueSchema; import org.apache.beam.sdk.schemas.annotations.DefaultSchema; import org.apache.beam.sdk.transforms.SerializableFunction; diff --git a/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/schematransform/SingleStoreSchemaTransformWriteConfiguration.java b/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/schematransform/SingleStoreSchemaTransformWriteConfiguration.java index b328065de87e..2903035e402e 100644 --- a/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/schematransform/SingleStoreSchemaTransformWriteConfiguration.java +++ b/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/schematransform/SingleStoreSchemaTransformWriteConfiguration.java @@ -19,9 +19,7 @@ import com.google.auto.value.AutoValue; import javax.annotation.Nullable; - import org.apache.beam.sdk.io.singlestore.SingleStoreIO; -import org.apache.beam.sdk.io.singlestore.schematransform.AutoValue_SingleStoreSchemaTransformWriteConfiguration; import org.apache.beam.sdk.schemas.AutoValueSchema; import org.apache.beam.sdk.schemas.annotations.DefaultSchema; import org.apache.beam.sdk.transforms.SerializableFunction; From 04432dd9415ba00a1212df249227e3c4cd2889c4 Mon Sep 17 00:00:00 2001 From: Adalbert Makarovych Date: Mon, 28 Nov 2022 19:02:06 +0200 Subject: [PATCH 09/21] Added package-info file --- .../schematransform/package-info.java | 20 +++++++++++++++++++ 1 file changed, 20 insertions(+) create mode 100644 sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/schematransform/package-info.java diff --git a/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/schematransform/package-info.java b/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/schematransform/package-info.java new file mode 100644 index 000000000000..d71e5eb209e9 --- /dev/null +++ b/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/schematransform/package-info.java @@ -0,0 +1,20 @@ +/* + * 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. + */ + +/** SingleStoreIO SchemaTransforms. */ +package org.apache.beam.sdk.io.singlestore.schematransform; From 103e4a8b56540e7e603ea2294afba4ecdcdc737a Mon Sep 17 00:00:00 2001 From: Adalbert Makarovych Date: Tue, 6 Dec 2022 13:41:24 +0200 Subject: [PATCH 10/21] check point --- .../SingleStoreSchemaTransformReadProvider.java | 4 +--- .../SingleStoreSchemaTransformWriteProvider.java | 3 +-- 2 files changed, 2 insertions(+), 5 deletions(-) diff --git a/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/schematransform/SingleStoreSchemaTransformReadProvider.java b/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/schematransform/SingleStoreSchemaTransformReadProvider.java index 32eb114d2a55..e5f1753dfea7 100644 --- a/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/schematransform/SingleStoreSchemaTransformReadProvider.java +++ b/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/schematransform/SingleStoreSchemaTransformReadProvider.java @@ -34,8 +34,6 @@ */ public class SingleStoreSchemaTransformReadProvider extends TypedSchemaTransformProvider { - - private static final String API = "singlestore"; private static final String OUTPUT_TAG = "OUTPUT"; /** Returns the expected class of the configuration. */ @@ -53,7 +51,7 @@ protected SchemaTransform from(SingleStoreSchemaTransformReadConfiguration confi /** Implementation of the {@link TypedSchemaTransformProvider} identifier method. */ @Override public String identifier() { - return String.format("%s:read", API); + return "beam:schematransform:org.apache.beam:singlestore_read:v1"; } /** diff --git a/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/schematransform/SingleStoreSchemaTransformWriteProvider.java b/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/schematransform/SingleStoreSchemaTransformWriteProvider.java index 0b23fdd0e323..7618696591c1 100644 --- a/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/schematransform/SingleStoreSchemaTransformWriteProvider.java +++ b/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/schematransform/SingleStoreSchemaTransformWriteProvider.java @@ -38,7 +38,6 @@ public class SingleStoreSchemaTransformWriteProvider extends TypedSchemaTransformProvider { - private static final String API = "singlestore"; private static final String OUTPUT_TAG = "OUTPUT"; public static final String INPUT_TAG = "INPUT"; @@ -57,7 +56,7 @@ protected SchemaTransform from(SingleStoreSchemaTransformWriteConfiguration conf /** Implementation of the {@link TypedSchemaTransformProvider} identifier method. */ @Override public String identifier() { - return String.format("%s:write", API); + return "beam:schematransform:org.apache.beam:singlestore_write:v1"; } /** From e67f05591ed1412a05acb03ba46dbe4dbbd24065 Mon Sep 17 00:00:00 2001 From: Adalbert Makarovych Date: Tue, 6 Dec 2022 15:51:53 +0200 Subject: [PATCH 11/21] check point --- .../sdk/io/singlestore/SingleStoreIO.java | 16 -- ...StoreSchemaTransformReadConfiguration.java | 5 + ...ingleStoreSchemaTransformReadProvider.java | 58 +++++-- ...nsformReadWithPartitionsConfiguration.java | 85 ---------- ...maTransformReadWithPartitionsProvider.java | 150 ------------------ ...ngleStoreSchemaTransformWriteProvider.java | 2 +- .../SingleStoreIOSchemaTransformIT.java | 10 +- 7 files changed, 52 insertions(+), 274 deletions(-) delete mode 100644 sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/schematransform/SingleStoreSchemaTransformReadWithPartitionsConfiguration.java delete mode 100644 sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/schematransform/SingleStoreSchemaTransformReadWithPartitionsProvider.java diff --git a/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreIO.java b/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreIO.java index 0ca93193bee6..079293f89e8b 100644 --- a/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreIO.java +++ b/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreIO.java @@ -897,8 +897,6 @@ public abstract static class ReadWithPartitionsRows extends PTransform expand(PBegin input) { DataSourceConfiguration dataSourceConfiguration = getDataSourceConfiguration(); Preconditions.checkArgumentNotNull( dataSourceConfiguration, "withDataSourceConfiguration() is required"); String actualQuery = SingleStoreUtil.getSelectQuery(getTable(), getQuery()); - Integer initialNumReaders = getInitialNumReaders(); SingleStoreDefaultRowMapper rowMapper = getRowMapper(dataSourceConfiguration, actualQuery); @@ -953,10 +942,6 @@ public PCollection expand(PBegin input) { .withRowMapper(rowMapper) .withCoder(RowCoder.of(rowMapper.getSchema())); - if (initialNumReaders != null) { - readWithPartitions = readWithPartitions.withInitialNumReaders(initialNumReaders); - } - PCollection output = input.apply(readWithPartitions); return output.setRowSchema(rowMapper.getSchema()); @@ -969,7 +954,6 @@ public void populateDisplayData(DisplayData.Builder builder) { DataSourceConfiguration.populateDisplayData(getDataSourceConfiguration(), builder); builder.addIfNotNull(DisplayData.item("query", getQuery())); builder.addIfNotNull(DisplayData.item("table", getTable())); - builder.addIfNotNull(DisplayData.item("initialNumReaders", getInitialNumReaders())); } } diff --git a/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/schematransform/SingleStoreSchemaTransformReadConfiguration.java b/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/schematransform/SingleStoreSchemaTransformReadConfiguration.java index d2d21c35d8fa..6951e9b31837 100644 --- a/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/schematransform/SingleStoreSchemaTransformReadConfiguration.java +++ b/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/schematransform/SingleStoreSchemaTransformReadConfiguration.java @@ -63,6 +63,9 @@ public Row toBeamRow() { @Nullable public abstract Boolean getOutputParallelization(); + @Nullable + public abstract Boolean getWithPartitions(); + @AutoValue.Builder public abstract static class Builder { @@ -74,6 +77,8 @@ public abstract static class Builder { public abstract Builder setOutputParallelization(Boolean value); + public abstract Builder setWithPartitions(Boolean value); + /** Builds the {@link SingleStoreSchemaTransformReadConfiguration} configuration. */ public abstract SingleStoreSchemaTransformReadConfiguration build(); } diff --git a/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/schematransform/SingleStoreSchemaTransformReadProvider.java b/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/schematransform/SingleStoreSchemaTransformReadProvider.java index e5f1753dfea7..3e274b285bf2 100644 --- a/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/schematransform/SingleStoreSchemaTransformReadProvider.java +++ b/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/schematransform/SingleStoreSchemaTransformReadProvider.java @@ -27,6 +27,7 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionRowTuple; import org.apache.beam.sdk.values.Row; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions; /** * An implementation of {@link TypedSchemaTransformProvider} for SingleStoreDB read jobs configured @@ -116,29 +117,54 @@ public PCollectionRowTuple expand(PCollectionRowTuple input) { String table = configuration.getTable(); String query = configuration.getQuery(); Boolean outputParallelization = configuration.getOutputParallelization(); + Boolean withPartitions = configuration.getWithPartitions(); - SingleStoreIO.ReadRows read = SingleStoreIO.readRows(); + Preconditions.checkArgument(!(outputParallelization != null && withPartitions != null && withPartitions), + "outputParallelization parameter is not supported for partitioned read"); - if (dataSourceConfiguration != null) { - read = read.withDataSourceConfiguration(dataSourceConfiguration); - } + if (withPartitions != null && withPartitions) { + SingleStoreIO.ReadWithPartitionsRows readWithPartitions = SingleStoreIO.readWithPartitionsRows(); - if (table != null) { - read = read.withTable(table); - } + if (dataSourceConfiguration != null) { + readWithPartitions = readWithPartitions.withDataSourceConfiguration(dataSourceConfiguration); + } - if (query != null) { - read = read.withQuery(query); - } + if (table != null && !table.isEmpty()) { + readWithPartitions = readWithPartitions.withTable(table); + } - if (outputParallelization != null) { - read = read.withOutputParallelization(outputParallelization); - } + if (query != null && !query.isEmpty()) { + readWithPartitions = readWithPartitions.withQuery(query); + } + + PCollection rows = input.getPipeline().apply(readWithPartitions); + Schema schema = rows.getSchema(); + + return PCollectionRowTuple.of(OUTPUT_TAG, rows.setRowSchema(schema)); + } else { + SingleStoreIO.ReadRows read = SingleStoreIO.readRows(); - PCollection rows = input.getPipeline().apply(read); - Schema schema = rows.getSchema(); + if (dataSourceConfiguration != null) { + read = read.withDataSourceConfiguration(dataSourceConfiguration); + } - return PCollectionRowTuple.of(OUTPUT_TAG, rows.setRowSchema(schema)); + if (table != null && !table.isEmpty()) { + read = read.withTable(table); + } + + if (query != null && !query.isEmpty()) { + read = read.withQuery(query); + } + + if (outputParallelization != null) { + read = read.withOutputParallelization(outputParallelization); + } + + PCollection rows = input.getPipeline().apply(read); + Schema schema = rows.getSchema(); + + return PCollectionRowTuple.of(OUTPUT_TAG, rows.setRowSchema(schema)); + } } } } diff --git a/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/schematransform/SingleStoreSchemaTransformReadWithPartitionsConfiguration.java b/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/schematransform/SingleStoreSchemaTransformReadWithPartitionsConfiguration.java deleted file mode 100644 index 67f0ab8f5b9a..000000000000 --- a/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/schematransform/SingleStoreSchemaTransformReadWithPartitionsConfiguration.java +++ /dev/null @@ -1,85 +0,0 @@ -/* - * 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.singlestore.schematransform; - -import com.google.auto.value.AutoValue; -import javax.annotation.Nullable; -import org.apache.beam.sdk.io.singlestore.SingleStoreIO; -import org.apache.beam.sdk.schemas.AutoValueSchema; -import org.apache.beam.sdk.schemas.annotations.DefaultSchema; -import org.apache.beam.sdk.transforms.SerializableFunction; -import org.apache.beam.sdk.values.Row; -import org.apache.beam.sdk.values.TypeDescriptor; - -/** - * Configuration for parallel reading from SignleStoreDB. - * - *

This class is meant to be used with {@link - * SingleStoreSchemaTransformReadWithPartitionsProvider}. - */ -@DefaultSchema(AutoValueSchema.class) -@AutoValue -public abstract class SingleStoreSchemaTransformReadWithPartitionsConfiguration { - - /** Instantiates a {@link SingleStoreSchemaTransformReadWithPartitionsConfiguration.Builder}. */ - public static Builder builder() { - return new AutoValue_SingleStoreSchemaTransformReadWithPartitionsConfiguration.Builder(); - } - - private static final AutoValueSchema AUTO_VALUE_SCHEMA = new AutoValueSchema(); - private static final TypeDescriptor - TYPE_DESCRIPTOR = - TypeDescriptor.of(SingleStoreSchemaTransformReadWithPartitionsConfiguration.class); - private static final SerializableFunction< - SingleStoreSchemaTransformReadWithPartitionsConfiguration, Row> - ROW_SERIALIZABLE_FUNCTION = AUTO_VALUE_SCHEMA.toRowFunction(TYPE_DESCRIPTOR); - - /** Serializes configuration to a {@link Row}. */ - public Row toBeamRow() { - return ROW_SERIALIZABLE_FUNCTION.apply(this); - } - - @Nullable - public abstract SingleStoreIO.DataSourceConfiguration getDataSourceConfiguration(); - - @Nullable - public abstract String getQuery(); - - @Nullable - public abstract String getTable(); - - @Nullable - public abstract Integer getInitialNumReaders(); - - @AutoValue.Builder - public abstract static class Builder { - - public abstract Builder setDataSourceConfiguration(SingleStoreIO.DataSourceConfiguration value); - - public abstract Builder setTable(String value); - - public abstract Builder setQuery(String value); - - public abstract Builder setInitialNumReaders(Integer value); - - /** - * Builds the {@link SingleStoreSchemaTransformReadWithPartitionsConfiguration} configuration. - */ - public abstract SingleStoreSchemaTransformReadWithPartitionsConfiguration build(); - } -} diff --git a/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/schematransform/SingleStoreSchemaTransformReadWithPartitionsProvider.java b/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/schematransform/SingleStoreSchemaTransformReadWithPartitionsProvider.java deleted file mode 100644 index 41f3f174c9ce..000000000000 --- a/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/schematransform/SingleStoreSchemaTransformReadWithPartitionsProvider.java +++ /dev/null @@ -1,150 +0,0 @@ -/* - * 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.singlestore.schematransform; - -import java.util.Collections; -import java.util.List; -import org.apache.beam.sdk.io.singlestore.SingleStoreIO; -import org.apache.beam.sdk.schemas.Schema; -import org.apache.beam.sdk.schemas.transforms.SchemaTransform; -import org.apache.beam.sdk.schemas.transforms.TypedSchemaTransformProvider; -import org.apache.beam.sdk.transforms.PTransform; -import org.apache.beam.sdk.values.PCollection; -import org.apache.beam.sdk.values.PCollectionRowTuple; -import org.apache.beam.sdk.values.Row; - -/** - * An implementation of {@link TypedSchemaTransformProvider} for SingleStoreDB parallel read jobs - * configured using {@link SingleStoreSchemaTransformReadWithPartitionsConfiguration}. - */ -public class SingleStoreSchemaTransformReadWithPartitionsProvider - extends TypedSchemaTransformProvider< - SingleStoreSchemaTransformReadWithPartitionsConfiguration> { - - private static final String API = "singlestore"; - private static final String OUTPUT_TAG = "OUTPUT"; - - /** Returns the expected class of the configuration. */ - @Override - protected Class configurationClass() { - return SingleStoreSchemaTransformReadWithPartitionsConfiguration.class; - } - - /** Returns the expected {@link SchemaTransform} of the configuration. */ - @Override - protected SchemaTransform from( - SingleStoreSchemaTransformReadWithPartitionsConfiguration configuration) { - return new SingleStoreReadWithPartitionsSchemaTransform(configuration); - } - - /** Implementation of the {@link TypedSchemaTransformProvider} identifier method. */ - @Override - public String identifier() { - return String.format("%s:read-with-partitions", API); - } - - /** - * Implementation of the {@link TypedSchemaTransformProvider} inputCollectionNames method. Since - * no input is expected, this returns an empty list. - */ - @Override - public List inputCollectionNames() { - return Collections.emptyList(); - } - - /** - * Implementation of the {@link TypedSchemaTransformProvider} outputCollectionNames method. Since - * a single output is expected, this returns a list with a single name. - */ - @Override - public List outputCollectionNames() { - return Collections.singletonList(OUTPUT_TAG); - } - - /** - * An implementation of {@link SchemaTransform} for SingleStoreDB parallel read jobs configured - * using {@link SingleStoreSchemaTransformReadWithPartitionsConfiguration}. - */ - private static class SingleStoreReadWithPartitionsSchemaTransform implements SchemaTransform { - private final SingleStoreSchemaTransformReadWithPartitionsConfiguration configuration; - - SingleStoreReadWithPartitionsSchemaTransform( - SingleStoreSchemaTransformReadWithPartitionsConfiguration configuration) { - this.configuration = configuration; - } - - /** Implements {@link SchemaTransform} buildTransform method. */ - @Override - public PTransform buildTransform() { - return new PCollectionRowTupleTransform(configuration); - } - } - - /** - * An implementation of {@link PTransform} for SingleStoreDB read jobs configured using {@link - * SingleStoreSchemaTransformReadWithPartitionsConfiguration}. - */ - static class PCollectionRowTupleTransform - extends PTransform { - - private final SingleStoreSchemaTransformReadWithPartitionsConfiguration configuration; - - PCollectionRowTupleTransform( - SingleStoreSchemaTransformReadWithPartitionsConfiguration configuration) { - this.configuration = configuration; - } - - @Override - public PCollectionRowTuple expand(PCollectionRowTuple input) { - if (!input.getAll().isEmpty()) { - throw new IllegalArgumentException( - String.format( - "%s %s input is expected to be empty", - input.getClass().getSimpleName(), getClass().getSimpleName())); - } - SingleStoreIO.DataSourceConfiguration dataSourceConfiguration = - configuration.getDataSourceConfiguration(); - String table = configuration.getTable(); - String query = configuration.getQuery(); - Integer initialNumReaders = configuration.getInitialNumReaders(); - - SingleStoreIO.ReadWithPartitionsRows read = SingleStoreIO.readWithPartitionsRows(); - - if (dataSourceConfiguration != null) { - read = read.withDataSourceConfiguration(dataSourceConfiguration); - } - - if (table != null) { - read = read.withTable(table); - } - - if (query != null) { - read = read.withQuery(query); - } - - if (initialNumReaders != null) { - read = read.withInitialNumReaders(initialNumReaders); - } - - PCollection rows = input.getPipeline().apply(read); - Schema schema = rows.getSchema(); - - return PCollectionRowTuple.of(OUTPUT_TAG, rows.setRowSchema(schema)); - } - } -} diff --git a/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/schematransform/SingleStoreSchemaTransformWriteProvider.java b/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/schematransform/SingleStoreSchemaTransformWriteProvider.java index 7618696591c1..5b68e1e05c5e 100644 --- a/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/schematransform/SingleStoreSchemaTransformWriteProvider.java +++ b/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/schematransform/SingleStoreSchemaTransformWriteProvider.java @@ -127,7 +127,7 @@ public PCollectionRowTuple expand(PCollectionRowTuple input) { write = write.withDataSourceConfiguration(dataSourceConfiguration); } - if (table != null) { + if (table != null && !table.isEmpty()) { write = write.withTable(table); } diff --git a/sdks/java/io/singlestore/src/test/java/org/apache/beam/sdk/io/singlestore/SingleStoreIOSchemaTransformIT.java b/sdks/java/io/singlestore/src/test/java/org/apache/beam/sdk/io/singlestore/SingleStoreIOSchemaTransformIT.java index 619d3422b953..4ded4cd452a8 100644 --- a/sdks/java/io/singlestore/src/test/java/org/apache/beam/sdk/io/singlestore/SingleStoreIOSchemaTransformIT.java +++ b/sdks/java/io/singlestore/src/test/java/org/apache/beam/sdk/io/singlestore/SingleStoreIOSchemaTransformIT.java @@ -31,8 +31,6 @@ import org.apache.beam.sdk.io.common.TestRow; import org.apache.beam.sdk.io.singlestore.schematransform.SingleStoreSchemaTransformReadConfiguration; import org.apache.beam.sdk.io.singlestore.schematransform.SingleStoreSchemaTransformReadProvider; -import org.apache.beam.sdk.io.singlestore.schematransform.SingleStoreSchemaTransformReadWithPartitionsConfiguration; -import org.apache.beam.sdk.io.singlestore.schematransform.SingleStoreSchemaTransformReadWithPartitionsProvider; import org.apache.beam.sdk.io.singlestore.schematransform.SingleStoreSchemaTransformWriteConfiguration; import org.apache.beam.sdk.io.singlestore.schematransform.SingleStoreSchemaTransformWriteProvider; import org.apache.beam.sdk.schemas.Schema; @@ -218,13 +216,13 @@ private PipelineResult runRead() { } private PipelineResult runReadWithPartitions() { - SchemaTransformProvider provider = new SingleStoreSchemaTransformReadWithPartitionsProvider(); + SchemaTransformProvider provider = new SingleStoreSchemaTransformReadProvider(); - SingleStoreSchemaTransformReadWithPartitionsConfiguration configuration = - SingleStoreSchemaTransformReadWithPartitionsConfiguration.builder() + SingleStoreSchemaTransformReadConfiguration configuration = + SingleStoreSchemaTransformReadConfiguration.builder() .setDataSourceConfiguration(dataSourceConfiguration) .setTable(tableName) - .setInitialNumReaders(1) + .setWithPartitions(true) .build(); Row configurationRow = configuration.toBeamRow(); From 7be63d7a8651759e9695dcca7f1560104d157d6f Mon Sep 17 00:00:00 2001 From: Adalbert Makarovych Date: Tue, 6 Dec 2022 15:59:03 +0200 Subject: [PATCH 12/21] Resolved comments Added DefaultSchema for DataSourceConfiguration Changed URNs Added checks for empty strings Deleted ReadWithPartitions schema transform and added withPartitions options to Read schema transform --- .../java/org/apache/beam/sdk/io/singlestore/SingleStoreIO.java | 3 +++ 1 file changed, 3 insertions(+) diff --git a/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreIO.java b/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreIO.java index 079293f89e8b..6ae43cf48d54 100644 --- a/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreIO.java +++ b/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreIO.java @@ -41,6 +41,8 @@ import org.apache.beam.sdk.coders.RowCoder; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.io.range.OffsetRange; +import org.apache.beam.sdk.schemas.AutoValueSchema; +import org.apache.beam.sdk.schemas.annotations.DefaultSchema; import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.Filter; @@ -243,6 +245,7 @@ public interface UserDataMapper extends Serializable { * A POJO describing a SingleStoreDB {@link DataSource} by providing all properties needed to * create it. */ + @DefaultSchema(AutoValueSchema.class) @AutoValue public abstract static class DataSourceConfiguration implements Serializable { abstract @Nullable String getEndpoint(); From 266ccc098d4ea8bd9b75b8a4fb31909b804fd863 Mon Sep 17 00:00:00 2001 From: Adalbert Makarovych Date: Tue, 6 Dec 2022 16:09:07 +0200 Subject: [PATCH 13/21] Changed identation --- .../SingleStoreSchemaTransformReadProvider.java | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/schematransform/SingleStoreSchemaTransformReadProvider.java b/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/schematransform/SingleStoreSchemaTransformReadProvider.java index 3e274b285bf2..17db41ec42c7 100644 --- a/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/schematransform/SingleStoreSchemaTransformReadProvider.java +++ b/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/schematransform/SingleStoreSchemaTransformReadProvider.java @@ -119,14 +119,17 @@ public PCollectionRowTuple expand(PCollectionRowTuple input) { Boolean outputParallelization = configuration.getOutputParallelization(); Boolean withPartitions = configuration.getWithPartitions(); - Preconditions.checkArgument(!(outputParallelization != null && withPartitions != null && withPartitions), + Preconditions.checkArgument( + !(outputParallelization != null && withPartitions != null && withPartitions), "outputParallelization parameter is not supported for partitioned read"); if (withPartitions != null && withPartitions) { - SingleStoreIO.ReadWithPartitionsRows readWithPartitions = SingleStoreIO.readWithPartitionsRows(); + SingleStoreIO.ReadWithPartitionsRows readWithPartitions = + SingleStoreIO.readWithPartitionsRows(); if (dataSourceConfiguration != null) { - readWithPartitions = readWithPartitions.withDataSourceConfiguration(dataSourceConfiguration); + readWithPartitions = + readWithPartitions.withDataSourceConfiguration(dataSourceConfiguration); } if (table != null && !table.isEmpty()) { From b6b6af06f34ca44ec785a4d6712682a9f6f605b7 Mon Sep 17 00:00:00 2001 From: Adalbert Makarovych Date: Wed, 14 Dec 2022 14:08:19 +0200 Subject: [PATCH 14/21] Fixed build by adding a cast --- .../sdk/io/singlestore/SingleStoreDefaultUserDataMapper.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreDefaultUserDataMapper.java b/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreDefaultUserDataMapper.java index fd3fb8c4c380..e83bf19b7095 100644 --- a/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreDefaultUserDataMapper.java +++ b/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreDefaultUserDataMapper.java @@ -33,7 +33,7 @@ final class SingleStoreDefaultUserDataMapper implements SingleStoreIO.UserDataMa DateTimeFormat.forPattern("yyyy-MM-DD' 'HH:mm:ss.SSS"); private String convertLogicalTypeFieldToString(Schema.FieldType type, Object value) { - Schema.LogicalType logicalType = type.getLogicalType(); + Schema.LogicalType logicalType = (Schema.LogicalType) type.getLogicalType(); if (logicalType == null) { throw new UnsupportedOperationException("Failed to extract logical type"); } From 3ffaa49c3217386291fa163d0a868a686b329ac6 Mon Sep 17 00:00:00 2001 From: Adalbert Makarovych Date: Wed, 14 Dec 2022 14:16:52 +0200 Subject: [PATCH 15/21] Reformatted code --- .../sdk/io/singlestore/SingleStoreDefaultUserDataMapper.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreDefaultUserDataMapper.java b/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreDefaultUserDataMapper.java index e83bf19b7095..27b5e1e2b5e6 100644 --- a/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreDefaultUserDataMapper.java +++ b/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreDefaultUserDataMapper.java @@ -33,7 +33,8 @@ final class SingleStoreDefaultUserDataMapper implements SingleStoreIO.UserDataMa DateTimeFormat.forPattern("yyyy-MM-DD' 'HH:mm:ss.SSS"); private String convertLogicalTypeFieldToString(Schema.FieldType type, Object value) { - Schema.LogicalType logicalType = (Schema.LogicalType) type.getLogicalType(); + Schema.LogicalType logicalType = + (Schema.LogicalType) type.getLogicalType(); if (logicalType == null) { throw new UnsupportedOperationException("Failed to extract logical type"); } From faa24169e6df6deb9d03b0a74be17df4f40fb1c7 Mon Sep 17 00:00:00 2001 From: Adalbert Makarovych Date: Wed, 14 Dec 2022 17:09:50 +0200 Subject: [PATCH 16/21] Added an assertion that convertLogicalTypeFieldToString is called only with logical types --- .../sdk/io/singlestore/SingleStoreDefaultUserDataMapper.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreDefaultUserDataMapper.java b/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreDefaultUserDataMapper.java index 27b5e1e2b5e6..0edf1dc2412c 100644 --- a/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreDefaultUserDataMapper.java +++ b/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreDefaultUserDataMapper.java @@ -33,6 +33,8 @@ final class SingleStoreDefaultUserDataMapper implements SingleStoreIO.UserDataMa DateTimeFormat.forPattern("yyyy-MM-DD' 'HH:mm:ss.SSS"); private String convertLogicalTypeFieldToString(Schema.FieldType type, Object value) { + assert type.getTypeName().isLogicalType(); + Schema.LogicalType logicalType = (Schema.LogicalType) type.getLogicalType(); if (logicalType == null) { From bdc532956916c63cb9112561fa7d608875a3e179 Mon Sep 17 00:00:00 2001 From: Adalbert Makarovych Date: Thu, 15 Dec 2022 15:45:05 +0200 Subject: [PATCH 17/21] Refactored code to delete ReadRows and ReadRowsWithPartitions classes --- .../SingleStoreDefaultRowMapper.java | 45 +-- .../SingleStoreDefaultUserDataMapper.java | 4 + .../sdk/io/singlestore/SingleStoreIO.java | 284 ++++-------------- .../sdk/io/singlestore/SingleStoreUtil.java | 9 +- ...ingleStoreSchemaTransformReadProvider.java | 4 +- .../SingleStoreDefaultRowMapperTest.java | 15 +- .../io/singlestore/SingleStoreUtilTest.java | 16 +- 7 files changed, 121 insertions(+), 256 deletions(-) diff --git a/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreDefaultRowMapper.java b/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreDefaultRowMapper.java index a09d146ab088..72d5f3ffa28b 100644 --- a/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreDefaultRowMapper.java +++ b/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreDefaultRowMapper.java @@ -55,7 +55,9 @@ import java.util.Calendar; import java.util.List; import java.util.TimeZone; +import org.apache.beam.sdk.coders.RowCoder; import org.apache.beam.sdk.schemas.Schema; +import org.apache.beam.sdk.schemas.SchemaCoder; import org.apache.beam.sdk.schemas.logicaltypes.VariableBytes; import org.apache.beam.sdk.schemas.logicaltypes.VariableString; import org.apache.beam.sdk.values.Row; @@ -64,13 +66,14 @@ import org.joda.time.LocalDate; import org.joda.time.chrono.ISOChronology; -class SingleStoreDefaultRowMapper implements SingleStoreIO.RowMapper { - Schema schema; +/** RowMapper that maps {@link ResultSet} row to the {@link Row}. */ +class SingleStoreDefaultRowMapper + implements SingleStoreIO.RowMapperWithInit, SingleStoreIO.RowMapperWithCoder { + @Nullable Schema schema = null; + List converters = new ArrayList<>(); - List converters; - - public SingleStoreDefaultRowMapper(ResultSetMetaData metaData) throws SQLException { - converters = new ArrayList<>(); + @Override + public void init(ResultSetMetaData metaData) throws SQLException { for (int i = 0; i < metaData.getColumnCount(); i++) { converters.add(ResultSetFieldConverter.of(metaData.getColumnType(i + 1))); } @@ -82,15 +85,16 @@ public SingleStoreDefaultRowMapper(ResultSetMetaData metaData) throws SQLExcepti this.schema = schemaBuilder.build(); } - public Schema getSchema() { - return this.schema; - } - @Override public Row mapRow(ResultSet resultSet) throws Exception { + if (schema == null) { + throw new UnsupportedOperationException("mapRow is called before init"); + } + Row.Builder rowBuilder = Row.withSchema(schema); - for (int i = 0; i < schema.getFieldCount(); i++) { + int fieldCount = schema.getFieldCount(); + for (int i = 0; i < fieldCount; i++) { Object value = converters.get(i).getValue(resultSet, i + 1); if (resultSet.wasNull() || value == null) { @@ -103,6 +107,15 @@ public Row mapRow(ResultSet resultSet) throws Exception { return rowBuilder.build(); } + @Override + public SchemaCoder getCoder() throws Exception { + if (schema == null) { + throw new UnsupportedOperationException("getCoder is called before init"); + } + + return RowCoder.of(this.schema); + } + abstract static class ResultSetFieldConverter implements Serializable { abstract @Nullable Object getValue(ResultSet rs, Integer index) throws SQLException; @@ -271,14 +284,4 @@ Schema.FieldType getSchemaFieldType(ResultSetMetaData md, Integer index) { return Schema.FieldType.DATETIME; } } - - public static class SingleStoreDefaultRowMapperCreationException extends RuntimeException { - SingleStoreDefaultRowMapperCreationException(String message, Throwable cause) { - super(message, cause); - } - - SingleStoreDefaultRowMapperCreationException(String message) { - super(message); - } - } } diff --git a/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreDefaultUserDataMapper.java b/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreDefaultUserDataMapper.java index 0edf1dc2412c..f2e9b9e693e1 100644 --- a/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreDefaultUserDataMapper.java +++ b/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreDefaultUserDataMapper.java @@ -27,6 +27,10 @@ import org.joda.time.format.DateTimeFormat; import org.joda.time.format.DateTimeFormatter; +/** + * UserDataMapper that maps {@link Row} objects. ARRAYs, ITTERABLEs, MAPs and nested ROWs are not + * supported. + */ final class SingleStoreDefaultUserDataMapper implements SingleStoreIO.UserDataMapper { private final transient DateTimeFormatter formatter = diff --git a/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreIO.java b/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreIO.java index 6ae43cf48d54..de2a68052789 100644 --- a/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreIO.java +++ b/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreIO.java @@ -31,14 +31,12 @@ import java.sql.PreparedStatement; import java.sql.ResultSet; import java.sql.ResultSetMetaData; -import java.sql.SQLException; import java.sql.Statement; import java.util.ArrayList; import java.util.List; import javax.sql.DataSource; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.ListCoder; -import org.apache.beam.sdk.coders.RowCoder; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.io.range.OffsetRange; import org.apache.beam.sdk.schemas.AutoValueSchema; @@ -175,8 +173,11 @@ public static Read read() { } /** Read Beam {@link Row}s from a SingleStoreDB datasource. */ - public static ReadRows readRows() { - return new AutoValue_SingleStoreIO_ReadRows.Builder().setOutputParallelization(true).build(); + public static Read readRows() { + return new AutoValue_SingleStoreIO_Read.Builder() + .setRowMapper(new SingleStoreDefaultRowMapper()) + .setOutputParallelization(true) + .build(); } /** @@ -193,8 +194,10 @@ public static ReadWithPartitions readWithPartitions() { * Like {@link #readRows}, but executes multiple instances of the query on the same table for each * database partition. */ - public static ReadWithPartitionsRows readWithPartitionsRows() { - return new AutoValue_SingleStoreIO_ReadWithPartitionsRows.Builder().build(); + public static ReadWithPartitions readWithPartitionsRows() { + return new AutoValue_SingleStoreIO_ReadWithPartitions.Builder() + .setRowMapper(new SingleStoreDefaultRowMapper()) + .build(); } /** @@ -222,6 +225,19 @@ public interface RowMapper extends Serializable { T mapRow(ResultSet resultSet) throws Exception; } + /** + * A RowMapper that requires initialization. init method is called during pipeline construction + * time. + */ + public interface RowMapperWithInit extends RowMapper { + void init(ResultSetMetaData resultSetMetaData) throws Exception; + } + + /** A RowMapper that provides a Coder for resulting PCollection. */ + public interface RowMapperWithCoder extends RowMapper { + Coder getCoder() throws Exception; + } + /** * An interface used by the SingleStoreIO {@link Read} to set the parameters of the {@link * PreparedStatement}. @@ -370,8 +386,6 @@ public abstract static class Read extends PTransform> abstract @Nullable RowMapper getRowMapper(); - abstract @Nullable Coder getCoder(); - abstract Builder toBuilder(); @AutoValue.Builder @@ -389,8 +403,6 @@ abstract Builder setDataSourceConfiguration( abstract Builder setRowMapper(RowMapper rowMapper); - abstract Builder setCoder(Coder coder); - abstract Read build(); } @@ -428,11 +440,6 @@ public Read withRowMapper(RowMapper rowMapper) { return toBuilder().setRowMapper(rowMapper).build(); } - public Read withCoder(Coder coder) { - checkNotNull(coder, "coder can not be null"); - return toBuilder().setCoder(coder).build(); - } - @Override public PCollection expand(PBegin input) { DataSourceConfiguration dataSourceConfiguration = getDataSourceConfiguration(); @@ -442,9 +449,17 @@ public PCollection expand(PBegin input) { Preconditions.checkArgumentNotNull(rowMapper, "withRowMapper() is required"); String actualQuery = SingleStoreUtil.getSelectQuery(getTable(), getQuery()); + if (rowMapper instanceof RowMapperWithInit) { + try { + ((RowMapperWithInit) rowMapper) + .init(getResultSetMetadata(dataSourceConfiguration, actualQuery)); + } catch (Exception e) { + throw new SingleStoreRowMapperInitializationException(e); + } + } + Coder coder = SingleStoreUtil.inferCoder( - getCoder(), rowMapper, input.getPipeline().getCoderRegistry(), input.getPipeline().getSchemaRegistry(), @@ -469,6 +484,12 @@ public PCollection expand(PBegin input) { return output; } + public static class SingleStoreRowMapperInitializationException extends RuntimeException { + SingleStoreRowMapperInitializationException(Throwable cause) { + super("Failed to initialize RowMapper", cause); + } + } + private static class ReadFn extends DoFn { DataSourceConfiguration dataSourceConfiguration; String query; @@ -565,136 +586,6 @@ public void populateDisplayData(DisplayData.Builder builder) { } } - private static SingleStoreDefaultRowMapper getRowMapper( - DataSourceConfiguration dataSourceConfiguration, String query) { - try { - DataSource dataSource = dataSourceConfiguration.getDataSource(); - Connection conn = dataSource.getConnection(); - try { - PreparedStatement stmt = - conn.prepareStatement(String.format("SELECT * FROM (%s) LIMIT 0", query)); - try { - ResultSetMetaData md = stmt.getMetaData(); - if (md == null) { - throw new SingleStoreDefaultRowMapper.SingleStoreDefaultRowMapperCreationException( - "Failed to retrieve statement metadata"); - } - return new SingleStoreDefaultRowMapper(md); - } finally { - stmt.close(); - } - } finally { - conn.close(); - } - } catch (SQLException e) { - throw new SingleStoreDefaultRowMapper.SingleStoreDefaultRowMapperCreationException( - "Failed to create default row mapper", e); - } - } - - @AutoValue - public abstract static class ReadRows extends PTransform> { - abstract @Nullable DataSourceConfiguration getDataSourceConfiguration(); - - abstract @Nullable String getQuery(); - - abstract @Nullable String getTable(); - - abstract @Nullable StatementPreparator getStatementPreparator(); - - abstract @Nullable Boolean getOutputParallelization(); - - abstract Builder toBuilder(); - - @AutoValue.Builder - abstract static class Builder { - abstract Builder setDataSourceConfiguration(DataSourceConfiguration dataSourceConfiguration); - - abstract Builder setQuery(String query); - - abstract Builder setTable(String table); - - abstract Builder setStatementPreparator(StatementPreparator statementPreparator); - - abstract Builder setOutputParallelization(Boolean outputParallelization); - - abstract ReadRows build(); - } - - public ReadRows withDataSourceConfiguration(DataSourceConfiguration config) { - checkNotNull(config, "dataSourceConfiguration can not be null"); - return toBuilder().setDataSourceConfiguration(config).build(); - } - - public ReadRows withQuery(String query) { - checkNotNull(query, "query can not be null"); - return toBuilder().setQuery(query).build(); - } - - public ReadRows withTable(String table) { - checkNotNull(table, "table can not be null"); - return toBuilder().setTable(table).build(); - } - - public ReadRows withStatementPreparator(StatementPreparator statementPreparator) { - checkNotNull(statementPreparator, "statementPreparator can not be null"); - return toBuilder().setStatementPreparator(statementPreparator).build(); - } - - /** - * Whether to reshuffle the resulting PCollection so results are distributed to all workers. The - * default is to parallelize and should only be changed if this is known to be unnecessary. - */ - public ReadRows withOutputParallelization(Boolean outputParallelization) { - checkNotNull(outputParallelization, "outputParallelization can not be null"); - return toBuilder().setOutputParallelization(outputParallelization).build(); - } - - @Override - public PCollection expand(PBegin input) { - DataSourceConfiguration dataSourceConfiguration = getDataSourceConfiguration(); - Preconditions.checkArgumentNotNull( - dataSourceConfiguration, "withDataSourceConfiguration() is required"); - String actualQuery = SingleStoreUtil.getSelectQuery(getTable(), getQuery()); - Boolean outputParallelization = getOutputParallelization(); - StatementPreparator statementPreparator = getStatementPreparator(); - - SingleStoreDefaultRowMapper rowMapper = getRowMapper(dataSourceConfiguration, actualQuery); - - Read read = - SingleStoreIO.read() - .withDataSourceConfiguration(dataSourceConfiguration) - .withQuery(actualQuery) - .withRowMapper(rowMapper) - .withCoder(RowCoder.of(rowMapper.getSchema())); - - if (outputParallelization != null) { - read = read.withOutputParallelization(outputParallelization); - } - - if (statementPreparator != null) { - read = read.withStatementPreparator(statementPreparator); - } - - PCollection output = input.apply(read); - - return output.setRowSchema(rowMapper.getSchema()); - } - - @Override - public void populateDisplayData(DisplayData.Builder builder) { - super.populateDisplayData(builder); - - DataSourceConfiguration.populateDisplayData(getDataSourceConfiguration(), builder); - builder.addIfNotNull(DisplayData.item("query", getQuery())); - builder.addIfNotNull(DisplayData.item("table", getTable())); - builder.addIfNotNull( - DisplayData.item( - "statementPreparator", SingleStoreUtil.getClassNameOrNull(getStatementPreparator()))); - builder.addIfNotNull(DisplayData.item("outputParallelization", getOutputParallelization())); - } - } - /** * A {@link PTransform} for reading data from SingleStoreDB. It is used by {@link * SingleStoreIO#readWithPartitions()}. {@link ReadWithPartitions} is preferred over {@link Read} @@ -714,8 +605,6 @@ public abstract static class ReadWithPartitions extends PTransform toBuilder(); - abstract @Nullable Coder getCoder(); - @AutoValue.Builder abstract static class Builder { abstract Builder setDataSourceConfiguration( @@ -727,8 +616,6 @@ abstract Builder setDataSourceConfiguration( abstract Builder setRowMapper(RowMapper rowMapper); - abstract Builder setCoder(Coder coder); - abstract ReadWithPartitions build(); } @@ -752,11 +639,6 @@ public ReadWithPartitions withRowMapper(RowMapper rowMapper) { return toBuilder().setRowMapper(rowMapper).build(); } - public ReadWithPartitions withCoder(Coder coder) { - checkNotNull(coder, "coder can not be null"); - return toBuilder().setCoder(coder).build(); - } - @Override public PCollection expand(PBegin input) { DataSourceConfiguration dataSourceConfiguration = getDataSourceConfiguration(); @@ -771,9 +653,17 @@ public PCollection expand(PBegin input) { String actualQuery = SingleStoreUtil.getSelectQuery(getTable(), getQuery()); + if (rowMapper instanceof RowMapperWithInit) { + try { + ((RowMapperWithInit) rowMapper) + .init(getResultSetMetadata(dataSourceConfiguration, actualQuery)); + } catch (Exception e) { + throw new Read.SingleStoreRowMapperInitializationException(e); + } + } + Coder coder = SingleStoreUtil.inferCoder( - getCoder(), rowMapper, input.getPipeline().getCoderRegistry(), input.getPipeline().getSchemaRegistry(), @@ -892,71 +782,25 @@ public void populateDisplayData(DisplayData.Builder builder) { } } - @AutoValue - public abstract static class ReadWithPartitionsRows extends PTransform> { - abstract @Nullable DataSourceConfiguration getDataSourceConfiguration(); - - abstract @Nullable String getQuery(); - - abstract @Nullable String getTable(); - - abstract ReadWithPartitionsRows.Builder toBuilder(); - - @AutoValue.Builder - abstract static class Builder { - abstract ReadWithPartitionsRows.Builder setDataSourceConfiguration( - DataSourceConfiguration dataSourceConfiguration); - - abstract Builder setQuery(String query); - - abstract Builder setTable(String table); - - abstract ReadWithPartitionsRows build(); - } - - public ReadWithPartitionsRows withDataSourceConfiguration(DataSourceConfiguration config) { - checkNotNull(config, "dataSourceConfiguration can not be null"); - return toBuilder().setDataSourceConfiguration(config).build(); - } - - public ReadWithPartitionsRows withQuery(String query) { - checkNotNull(query, "query can not be null"); - return toBuilder().setQuery(query).build(); - } - - public ReadWithPartitionsRows withTable(String table) { - checkNotNull(table, "table can not be null"); - return toBuilder().setTable(table).build(); - } - - @Override - public PCollection expand(PBegin input) { - DataSourceConfiguration dataSourceConfiguration = getDataSourceConfiguration(); - Preconditions.checkArgumentNotNull( - dataSourceConfiguration, "withDataSourceConfiguration() is required"); - String actualQuery = SingleStoreUtil.getSelectQuery(getTable(), getQuery()); - - SingleStoreDefaultRowMapper rowMapper = getRowMapper(dataSourceConfiguration, actualQuery); - - ReadWithPartitions readWithPartitions = - SingleStoreIO.readWithPartitions() - .withDataSourceConfiguration(dataSourceConfiguration) - .withQuery(actualQuery) - .withRowMapper(rowMapper) - .withCoder(RowCoder.of(rowMapper.getSchema())); - - PCollection output = input.apply(readWithPartitions); - - return output.setRowSchema(rowMapper.getSchema()); - } - - @Override - public void populateDisplayData(DisplayData.Builder builder) { - super.populateDisplayData(builder); + private static ResultSetMetaData getResultSetMetadata( + DataSourceConfiguration dataSourceConfiguration, String query) throws Exception { + DataSource dataSource = dataSourceConfiguration.getDataSource(); + Connection conn = dataSource.getConnection(); + try { + PreparedStatement stmt = + conn.prepareStatement(String.format("SELECT * FROM (%s) LIMIT 0", query)); + try { + ResultSetMetaData md = stmt.getMetaData(); + if (md == null) { + throw new Exception("ResultSetMetaData is null"); + } - DataSourceConfiguration.populateDisplayData(getDataSourceConfiguration(), builder); - builder.addIfNotNull(DisplayData.item("query", getQuery())); - builder.addIfNotNull(DisplayData.item("table", getTable())); + return md; + } finally { + stmt.close(); + } + } finally { + conn.close(); } } diff --git a/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreUtil.java b/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreUtil.java index 3b04cb9f8c0a..f64885e723fd 100644 --- a/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreUtil.java +++ b/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreUtil.java @@ -38,13 +38,16 @@ public static String escapeString(String identifier) { } public static Coder inferCoder( - @Nullable Coder coder, SingleStoreIO.RowMapper rowMapper, CoderRegistry registry, SchemaRegistry schemaRegistry, Logger log) { - if (coder != null) { - return coder; + if (rowMapper instanceof SingleStoreIO.RowMapperWithCoder) { + try { + return ((SingleStoreIO.RowMapperWithCoder) rowMapper).getCoder(); + } catch (Exception e) { + log.warn("Unable to infer a coder from RowMapper. Attempting to infer a coder from type."); + } } TypeDescriptor outputType = diff --git a/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/schematransform/SingleStoreSchemaTransformReadProvider.java b/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/schematransform/SingleStoreSchemaTransformReadProvider.java index 17db41ec42c7..601fd1a89d11 100644 --- a/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/schematransform/SingleStoreSchemaTransformReadProvider.java +++ b/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/schematransform/SingleStoreSchemaTransformReadProvider.java @@ -124,7 +124,7 @@ public PCollectionRowTuple expand(PCollectionRowTuple input) { "outputParallelization parameter is not supported for partitioned read"); if (withPartitions != null && withPartitions) { - SingleStoreIO.ReadWithPartitionsRows readWithPartitions = + SingleStoreIO.ReadWithPartitions readWithPartitions = SingleStoreIO.readWithPartitionsRows(); if (dataSourceConfiguration != null) { @@ -145,7 +145,7 @@ public PCollectionRowTuple expand(PCollectionRowTuple input) { return PCollectionRowTuple.of(OUTPUT_TAG, rows.setRowSchema(schema)); } else { - SingleStoreIO.ReadRows read = SingleStoreIO.readRows(); + SingleStoreIO.Read read = SingleStoreIO.readRows(); if (dataSourceConfiguration != null) { read = read.withDataSourceConfiguration(dataSourceConfiguration); diff --git a/sdks/java/io/singlestore/src/test/java/org/apache/beam/sdk/io/singlestore/SingleStoreDefaultRowMapperTest.java b/sdks/java/io/singlestore/src/test/java/org/apache/beam/sdk/io/singlestore/SingleStoreDefaultRowMapperTest.java index 87afb6069ee0..54de09a4e953 100644 --- a/sdks/java/io/singlestore/src/test/java/org/apache/beam/sdk/io/singlestore/SingleStoreDefaultRowMapperTest.java +++ b/sdks/java/io/singlestore/src/test/java/org/apache/beam/sdk/io/singlestore/SingleStoreDefaultRowMapperTest.java @@ -63,8 +63,9 @@ public void testEmptyRow() throws Exception { Mockito.when(md.getColumnCount()).thenReturn(0); ResultSet res = Mockito.mock(ResultSet.class); - SingleStoreDefaultRowMapper mapper = new SingleStoreDefaultRowMapper(md); - Schema s = mapper.getSchema(); + SingleStoreDefaultRowMapper mapper = new SingleStoreDefaultRowMapper(); + mapper.init(md); + Schema s = mapper.getCoder().getSchema(); Row r = mapper.mapRow(res); assertEquals(0, s.getFieldCount()); @@ -128,8 +129,9 @@ public void testAllDataTypes() throws Exception { Mockito.when(res.wasNull()).thenReturn(false); - SingleStoreDefaultRowMapper mapper = new SingleStoreDefaultRowMapper(md); - Schema s = mapper.getSchema(); + SingleStoreDefaultRowMapper mapper = new SingleStoreDefaultRowMapper(); + mapper.init(md); + Schema s = mapper.getCoder().getSchema(); Row r = mapper.mapRow(res); assertEquals(17, s.getFieldCount()); @@ -235,8 +237,9 @@ public void testNullValues() throws Exception { Mockito.when(res.wasNull()).thenReturn(true); - SingleStoreDefaultRowMapper mapper = new SingleStoreDefaultRowMapper(md); - Schema s = mapper.getSchema(); + SingleStoreDefaultRowMapper mapper = new SingleStoreDefaultRowMapper(); + mapper.init(md); + Schema s = mapper.getCoder().getSchema(); Row r = mapper.mapRow(res); assertEquals(18, s.getFieldCount()); diff --git a/sdks/java/io/singlestore/src/test/java/org/apache/beam/sdk/io/singlestore/SingleStoreUtilTest.java b/sdks/java/io/singlestore/src/test/java/org/apache/beam/sdk/io/singlestore/SingleStoreUtilTest.java index 2ea7c1095cc3..6f50d419368e 100644 --- a/sdks/java/io/singlestore/src/test/java/org/apache/beam/sdk/io/singlestore/SingleStoreUtilTest.java +++ b/sdks/java/io/singlestore/src/test/java/org/apache/beam/sdk/io/singlestore/SingleStoreUtilTest.java @@ -79,13 +79,21 @@ public TestRow mapRow(ResultSet resultSet) throws Exception { } } + private static class TestRowMapperWithCoder extends TestRowMapper + implements SingleStoreIO.RowMapperWithCoder { + @Override + public Coder getCoder() throws Exception { + return SerializableCoder.of(TestRow.class); + } + } + @Test - public void testInferCoderFromCoder() { + public void testInferCoderFromRowMapper() { SchemaRegistry sr = SchemaRegistry.createDefault(); CoderRegistry cr = CoderRegistry.createDefault(); Coder c = SerializableCoder.of(TestRow.class); - assertEquals(c, SingleStoreUtil.inferCoder(c, new TestRowMapper(), cr, sr, LOG)); + assertEquals(c, SingleStoreUtil.inferCoder(new TestRowMapperWithCoder(), cr, sr, LOG)); } @Test @@ -95,7 +103,7 @@ public void testInferCoderFromSchemaRegistry() { Coder c = SerializableCoder.of(TestRow.class); cr.registerCoderForClass(TestRow.class, c); - assertEquals(c, SingleStoreUtil.inferCoder(null, new TestRowMapper(), cr, sr, LOG)); + assertEquals(c, SingleStoreUtil.inferCoder(new TestRowMapper(), cr, sr, LOG)); } @Test @@ -105,7 +113,7 @@ public void testInferCoderFromCoderRegistry() throws NoSuchSchemaException { sr.registerPOJO(TestRow.class); Coder c = sr.getSchemaCoder(TestRow.class); - assertEquals(c, SingleStoreUtil.inferCoder(null, new TestRowMapper(), cr, sr, LOG)); + assertEquals(c, SingleStoreUtil.inferCoder(new TestRowMapper(), cr, sr, LOG)); } @Test From b9d253ec6bf81a52306b4f5b9580a5aea1b57b2c Mon Sep 17 00:00:00 2001 From: AdalbertMemSQL <55380838+AdalbertMemSQL@users.noreply.github.com> Date: Thu, 15 Dec 2022 18:22:07 +0200 Subject: [PATCH 18/21] Update .test-infra/jenkins/job_PostCommit_Java_SingleStoreIO_IT.groovy Co-authored-by: Ahmed Abualsaud <65791736+ahmedabu98@users.noreply.github.com> --- .test-infra/jenkins/job_PostCommit_Java_SingleStoreIO_IT.groovy | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.test-infra/jenkins/job_PostCommit_Java_SingleStoreIO_IT.groovy b/.test-infra/jenkins/job_PostCommit_Java_SingleStoreIO_IT.groovy index 23664ee4ba79..4a392c065b38 100644 --- a/.test-infra/jenkins/job_PostCommit_Java_SingleStoreIO_IT.groovy +++ b/.test-infra/jenkins/job_PostCommit_Java_SingleStoreIO_IT.groovy @@ -41,7 +41,7 @@ PostcommitJobBuilder.postCommitJob(jobName, description('Runs the Java SingleStoreIO Integration Test.') // Set common parameters. - commonJobProperties.setTopLevelMainJobProperties(delegate) + common.setTopLevelMainJobProperties(delegate) // Deploy SingleStoreDB cluster String namespace = common.getKubernetesNamespace(jobName) From 7c9b4f3f6db7018c19a711f4138e60623de70956 Mon Sep 17 00:00:00 2001 From: Adalbert Makarovych Date: Thu, 15 Dec 2022 23:16:09 +0200 Subject: [PATCH 19/21] Fixed bug where env variable name was used instead of the value --- .test-infra/jenkins/job_PostCommit_Java_SingleStoreIO_IT.groovy | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.test-infra/jenkins/job_PostCommit_Java_SingleStoreIO_IT.groovy b/.test-infra/jenkins/job_PostCommit_Java_SingleStoreIO_IT.groovy index 4a392c065b38..f43da5a0531a 100644 --- a/.test-infra/jenkins/job_PostCommit_Java_SingleStoreIO_IT.groovy +++ b/.test-infra/jenkins/job_PostCommit_Java_SingleStoreIO_IT.groovy @@ -64,7 +64,7 @@ PostcommitJobBuilder.postCommitJob(jobName, tempRoot : 'gs://temp-storage-for-perf-tests', project : 'apache-beam-testing', runner : 'DataflowRunner', - singleStoreServerName : singlestoreHostName, + singleStoreServerName : "\$${singlestoreHostName}", singleStoreUsername : "admin", singleStorePassword : "secretpass", singleStorePort: "3306", From 891508b97e2ca5262056abad24187a119dc65b1f Mon Sep 17 00:00:00 2001 From: Adalbert Makarovych Date: Thu, 15 Dec 2022 23:18:13 +0200 Subject: [PATCH 20/21] Changed to use checkArgument instead of assert --- .../sdk/io/singlestore/SingleStoreDefaultUserDataMapper.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreDefaultUserDataMapper.java b/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreDefaultUserDataMapper.java index f2e9b9e693e1..a7ca595d699d 100644 --- a/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreDefaultUserDataMapper.java +++ b/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreDefaultUserDataMapper.java @@ -17,6 +17,8 @@ */ package org.apache.beam.sdk.io.singlestore; +import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument; + import java.math.BigDecimal; import java.nio.charset.StandardCharsets; import java.util.ArrayList; @@ -37,7 +39,7 @@ final class SingleStoreDefaultUserDataMapper implements SingleStoreIO.UserDataMa DateTimeFormat.forPattern("yyyy-MM-DD' 'HH:mm:ss.SSS"); private String convertLogicalTypeFieldToString(Schema.FieldType type, Object value) { - assert type.getTypeName().isLogicalType(); + checkArgument(type.getTypeName().isLogicalType(), ""); Schema.LogicalType logicalType = (Schema.LogicalType) type.getLogicalType(); From 98a25bd8748a4111326325575573d0b57c4b2bad Mon Sep 17 00:00:00 2001 From: Adalbert Makarovych Date: Mon, 19 Dec 2022 23:07:04 +0200 Subject: [PATCH 21/21] Added appropriate error message --- .../sdk/io/singlestore/SingleStoreDefaultUserDataMapper.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreDefaultUserDataMapper.java b/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreDefaultUserDataMapper.java index a7ca595d699d..2035ce0554f1 100644 --- a/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreDefaultUserDataMapper.java +++ b/sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreDefaultUserDataMapper.java @@ -39,7 +39,9 @@ final class SingleStoreDefaultUserDataMapper implements SingleStoreIO.UserDataMa DateTimeFormat.forPattern("yyyy-MM-DD' 'HH:mm:ss.SSS"); private String convertLogicalTypeFieldToString(Schema.FieldType type, Object value) { - checkArgument(type.getTypeName().isLogicalType(), ""); + checkArgument( + type.getTypeName().isLogicalType(), + "convertLogicalTypeFieldToString accepts only logical types"); Schema.LogicalType logicalType = (Schema.LogicalType) type.getLogicalType();