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 1 commit
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 @@ -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;
Expand All @@ -64,13 +66,14 @@
import org.joda.time.LocalDate;
import org.joda.time.chrono.ISOChronology;

class SingleStoreDefaultRowMapper implements SingleStoreIO.RowMapper<Row> {
Schema schema;
/** 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<>();

List<ResultSetFieldConverter> 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)));
}
Expand All @@ -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) {
Expand All @@ -103,6 +107,15 @@ public Row mapRow(ResultSet resultSet) throws Exception {
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;

Expand Down Expand Up @@ -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);
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -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<Row> {

private final transient DateTimeFormatter formatter =
Expand Down
Loading