ahmedabu98 commented on code in PR #24290: URL: https://github.com/apache/beam/pull/24290#discussion_r1048556442
########## sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/SingleStoreDefaultUserDataMapper.java: ########## @@ -0,0 +1,104 @@ +/* + * 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<Row> { + + private final transient DateTimeFormatter formatter = + DateTimeFormat.forPattern("yyyy-MM-DD' 'HH:mm:ss.SSS"); + + private String convertLogicalTypeFieldToString(Schema.FieldType type, Object value) { + Schema.LogicalType<Object, Object> logicalType = + (Schema.LogicalType<Object, Object>) type.getLogicalType(); + if (logicalType == null) { + throw new UnsupportedOperationException("Failed to extract logical type"); + } Review Comment: You can make use of [`FieldType::isLogicalType`](https://github.com/apache/beam/blob/3ffaa49c3217386291fa163d0a868a686b329ac6/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/Schema.java#L643) as a check here. ########## sdks/java/io/singlestore/src/main/java/org/apache/beam/sdk/io/singlestore/schematransform/SingleStoreSchemaTransformReadConfiguration.java: ########## @@ -0,0 +1,80 @@ +/* + * 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 reading from SignleStoreDB. + * + * <p>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<SingleStoreSchemaTransformReadConfiguration> TYPE_DESCRIPTOR = + TypeDescriptor.of(SingleStoreSchemaTransformReadConfiguration.class); + private static final SerializableFunction<SingleStoreSchemaTransformReadConfiguration, 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 Boolean getOutputParallelization(); + + @AutoValue.Builder + public abstract static class Builder { + + public abstract Builder setDataSourceConfiguration(SingleStoreIO.DataSourceConfiguration value); Review Comment: Just tested this out locally with a simple configuration that had a POJO field and generating a schema worked fine with just the `@AutoValue` decoration: ``` Field{name=field1, description=, type=STRING NOT NULL, options={{}}} Field{name=field2, description=, type=INT32 NOT NULL, options={{}}} Field{name=pojoField, description=, type=ROW< pojoField1 STRING NOT NULL, pojoField2 INT32 NOT NULL > NOT NULL, options={{}}} ``` ########## .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) Review Comment: 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](https://github.com/apache/beam/blob/ef5351ad50a817498ea9e34a7c514dd9d60fb143/.test-infra/jenkins/job_PostCommit_Java.groovy#L32). -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
