-
Notifications
You must be signed in to change notification settings - Fork 4.3k
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
Changes from 17 commits
Commits
Show all changes
21 commits
Select commit
Hold shift + click to select a range
e97f319
Implemented SchemaTransforms for SingleStoreIO
AdalbertMemSQL 69a1d21
Fixed nullable errors
AdalbertMemSQL 08b3132
Changed to don't use .* form of import
AdalbertMemSQL caf46da
Changed formatter field to be transient
AdalbertMemSQL 302ceac
Nit reformatting
AdalbertMemSQL b5e9543
Fixed bugs in tests
AdalbertMemSQL 4e2e7bc
Moved schema transform classes to the separate folder
AdalbertMemSQL d59828a
Removed unused imports
AdalbertMemSQL 04432dd
Added package-info file
AdalbertMemSQL 103e4a8
check point
AdalbertMemSQL e67f055
check point
AdalbertMemSQL 7be63d7
Resolved comments
AdalbertMemSQL 266ccc0
Changed identation
AdalbertMemSQL b6b6af0
Fixed build by adding a cast
AdalbertMemSQL 3ffaa49
Reformatted code
AdalbertMemSQL faa2416
Added an assertion that convertLogicalTypeFieldToString is called onl…
AdalbertMemSQL bdc5329
Refactored code to delete ReadRows and ReadRowsWithPartitions classes
AdalbertMemSQL b9d253e
Update .test-infra/jenkins/job_PostCommit_Java_SingleStoreIO_IT.groovy
AdalbertMemSQL 7c9b4f3
Fixed bug where env variable name was used instead of the value
AdalbertMemSQL 891508b
Changed to use checkArgument instead of assert
AdalbertMemSQL 98a25bd
Added appropriate error message
AdalbertMemSQL File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
86 changes: 86 additions & 0 deletions
86
.test-infra/jenkins/job_PostCommit_Java_SingleStoreIO_IT.groovy
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
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. | ||
commonJobProperties.setTopLevelMainJobProperties(delegate) | ||
AdalbertMemSQL marked this conversation as resolved.
Show resolved
Hide resolved
|
||
|
||
// 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") | ||
} | ||
} | ||
} |
287 changes: 287 additions & 0 deletions
287
...lestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreDefaultRowMapper.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
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; | ||
} | ||
} | ||
} |
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Can you set a timeout here? This is not to set a strict time limit for the job, but more to catch runaway jobs, see example here.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Looks like the default timeout is already set to 100