Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Implemented SchemaTransforms for SingleStoreIO #24290

Merged
merged 21 commits into from
Dec 19, 2022
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
Show all changes
21 commits
Select commit Hold shift + click to select a range
e97f319
Implemented SchemaTransforms for SingleStoreIO
AdalbertMemSQL Nov 14, 2022
69a1d21
Fixed nullable errors
AdalbertMemSQL Nov 24, 2022
08b3132
Changed to don't use .* form of import
AdalbertMemSQL Nov 24, 2022
caf46da
Changed formatter field to be transient
AdalbertMemSQL Nov 25, 2022
302ceac
Nit reformatting
AdalbertMemSQL Nov 25, 2022
b5e9543
Fixed bugs in tests
AdalbertMemSQL Nov 25, 2022
4e2e7bc
Moved schema transform classes to the separate folder
AdalbertMemSQL Nov 28, 2022
d59828a
Removed unused imports
AdalbertMemSQL Nov 28, 2022
04432dd
Added package-info file
AdalbertMemSQL Nov 28, 2022
103e4a8
check point
AdalbertMemSQL Dec 6, 2022
e67f055
check point
AdalbertMemSQL Dec 6, 2022
7be63d7
Resolved comments
AdalbertMemSQL Dec 6, 2022
266ccc0
Changed identation
AdalbertMemSQL Dec 6, 2022
b6b6af0
Fixed build by adding a cast
AdalbertMemSQL Dec 14, 2022
3ffaa49
Reformatted code
AdalbertMemSQL Dec 14, 2022
faa2416
Added an assertion that convertLogicalTypeFieldToString is called onl…
AdalbertMemSQL Dec 14, 2022
bdc5329
Refactored code to delete ReadRows and ReadRowsWithPartitions classes
AdalbertMemSQL Dec 15, 2022
b9d253e
Update .test-infra/jenkins/job_PostCommit_Java_SingleStoreIO_IT.groovy
AdalbertMemSQL Dec 15, 2022
7c9b4f3
Fixed bug where env variable name was used instead of the value
AdalbertMemSQL Dec 15, 2022
891508b
Changed to use checkArgument instead of assert
AdalbertMemSQL Dec 15, 2022
98a25bd
Added appropriate error message
AdalbertMemSQL Dec 19, 2022
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -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")
}
}
}
86 changes: 86 additions & 0 deletions .test-infra/jenkins/job_PostCommit_Java_SingleStoreIO_IT.groovy
Original file line number Diff line number Diff line change
@@ -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.
common.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")
}
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,287 @@
/*
* 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.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;
import org.checkerframework.checker.nullness.qual.Nullable;
import org.joda.time.DateTime;
import org.joda.time.LocalDate;
import org.joda.time.chrono.ISOChronology;

/** RowMapper that maps {@link ResultSet} row to the {@link Row}. */
class SingleStoreDefaultRowMapper
implements SingleStoreIO.RowMapperWithInit<Row>, SingleStoreIO.RowMapperWithCoder<Row> {
@Nullable Schema schema = null;
List<ResultSetFieldConverter> 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)));
}

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();
}

@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);

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) {
rowBuilder.addValue(null);
} else {
rowBuilder.addValue(value);
}
}

return rowBuilder.build();
}

@Override
public SchemaCoder<Row> 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;

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 {
@Nullable
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
@Nullable
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
@Nullable
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
@Nullable
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
@Nullable
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
@Nullable
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
@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);
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;
}
}
}
Loading