robinyqiu commented on a change in pull request #12202: URL: https://github.com/apache/beam/pull/12202#discussion_r456714938
########## File path: sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/pubsub/PubsubJsonTableProvider.java ########## @@ -17,63 +17,32 @@ */ package org.apache.beam.sdk.extensions.sql.meta.provider.pubsub; -import static org.apache.beam.sdk.util.RowJsonUtils.newObjectMapperWith; - -import com.alibaba.fastjson.JSONObject; -import com.fasterxml.jackson.core.JsonProcessingException; import com.google.auto.service.AutoService; import org.apache.beam.sdk.annotations.Experimental; import org.apache.beam.sdk.annotations.Internal; -import org.apache.beam.sdk.extensions.sql.meta.BeamSqlTable; -import org.apache.beam.sdk.extensions.sql.meta.Table; -import org.apache.beam.sdk.extensions.sql.meta.provider.InMemoryMetaTableProvider; -import org.apache.beam.sdk.extensions.sql.meta.provider.InvalidTableException; +import org.apache.beam.sdk.extensions.sql.meta.provider.SchemaCapableIOTableProviderWrapper; import org.apache.beam.sdk.extensions.sql.meta.provider.TableProvider; import org.apache.beam.sdk.io.gcp.pubsub.PubsubIO; import org.apache.beam.sdk.io.gcp.pubsub.PubsubSchemaCapableIOProvider; -import org.apache.beam.sdk.schemas.io.InvalidConfigurationException; -import org.apache.beam.sdk.schemas.io.InvalidSchemaException; -import org.apache.beam.sdk.schemas.io.SchemaIO; -import org.apache.beam.sdk.util.RowJson.RowJsonDeserializer; -import org.apache.beam.sdk.values.Row; +import org.apache.beam.sdk.schemas.io.SchemaIOProvider; /** - * {@link TableProvider} for {@link PubsubIOJsonTable} which wraps {@link PubsubIO} for consumption - * by Beam SQL. + * {@link TableProvider} for {@link PubsubIO} for consumption by Beam SQL. + * + * <p>Passes the {@link PubsubSchemaCapableIOProvider} to the generalized table provider wrapper, + * {@link SchemaCapableIOTableProviderWrapper}, for Pubsub specific behavior. */ @Internal @Experimental @AutoService(TableProvider.class) -public class PubsubJsonTableProvider extends InMemoryMetaTableProvider { - +public class PubsubJsonTableProvider extends SchemaCapableIOTableProviderWrapper { @Override - public String getTableType() { - return "pubsub"; + public SchemaIOProvider getSchemaIOProvider() { + return new PubsubSchemaCapableIOProvider(); } @Override - public BeamSqlTable buildBeamSqlTable(Table tableDefinition) { - JSONObject tableProperties = tableDefinition.getProperties(); - PubsubSchemaCapableIOProvider ioProvider = new PubsubSchemaCapableIOProvider(); - - try { - RowJsonDeserializer deserializer = - RowJsonDeserializer.forSchema(ioProvider.configurationSchema()) - .withNullBehavior(RowJsonDeserializer.NullBehavior.ACCEPT_MISSING_OR_NULL); - - Row configurationRow = - newObjectMapperWith(deserializer).readValue(tableProperties.toString(), Row.class); - - SchemaIO pubsubSchemaIO = - ioProvider.from( - tableDefinition.getLocation(), configurationRow, tableDefinition.getSchema()); - - return PubsubIOJsonTable.fromSchemaIO(pubsubSchemaIO); - } catch (InvalidConfigurationException | InvalidSchemaException e) { - throw new InvalidTableException(e.getMessage()); - } catch (JsonProcessingException e) { - throw new AssertionError( - "Failed to re-parse TBLPROPERTIES JSON " + tableProperties.toString()); - } + public String getTableType() { Review comment: Please add a TODO with JIRA link here such that people know this override is only a temporary solution, and we can remove it after the issue tracked by that JIRA is fixed. Like ``` // TODO[BEAM-?????]: remove this override after TableProvider problem is fixed ``` And we should do the same thing for all classes that extends `SchemaCapableIOTableProviderWrapper` ########## File path: sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/SchemaCapableIOTableProviderWrapper.java ########## @@ -0,0 +1,132 @@ +/* + * 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.extensions.sql.meta.provider; + +import static org.apache.beam.sdk.util.RowJsonUtils.newObjectMapperWith; + +import com.alibaba.fastjson.JSONObject; +import com.fasterxml.jackson.core.JsonProcessingException; +import java.io.Serializable; +import org.apache.beam.sdk.annotations.Experimental; +import org.apache.beam.sdk.annotations.Internal; +import org.apache.beam.sdk.extensions.sql.impl.BeamTableStatistics; +import org.apache.beam.sdk.extensions.sql.meta.BaseBeamTable; +import org.apache.beam.sdk.extensions.sql.meta.BeamSqlTable; +import org.apache.beam.sdk.extensions.sql.meta.Table; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.schemas.Schema; +import org.apache.beam.sdk.schemas.io.InvalidConfigurationException; +import org.apache.beam.sdk.schemas.io.InvalidSchemaException; +import org.apache.beam.sdk.schemas.io.SchemaIO; +import org.apache.beam.sdk.schemas.io.SchemaIOProvider; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.util.RowJson; +import org.apache.beam.sdk.values.PBegin; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.POutput; +import org.apache.beam.sdk.values.Row; + +/** + * A general {@link TableProvider} for IOs for consumption by Beam SQL. + * + * <p>Can create child classes for IOs to pass {@link #getSchemaIOProvider()} that is specific to + * the IO. + */ +@Internal +@Experimental +public abstract class SchemaCapableIOTableProviderWrapper extends InMemoryMetaTableProvider + implements Serializable { + public abstract SchemaIOProvider getSchemaIOProvider(); + + @Override + public String getTableType() { + return getSchemaIOProvider().identifier(); + } + + @Override + public BeamSqlTable buildBeamSqlTable(Table tableDefinition) { + JSONObject tableProperties = tableDefinition.getProperties(); + + try { + RowJson.RowJsonDeserializer deserializer = + RowJson.RowJsonDeserializer.forSchema(getSchemaIOProvider().configurationSchema()) + .withNullBehavior(RowJson.RowJsonDeserializer.NullBehavior.ACCEPT_MISSING_OR_NULL); + + Row configurationRow = + newObjectMapperWith(deserializer).readValue(tableProperties.toString(), Row.class); + + SchemaIO schemaIO = + getSchemaIOProvider() + .from(tableDefinition.getLocation(), configurationRow, tableDefinition.getSchema()); + + return new SchemaIOTableWrapper(schemaIO); + } catch (InvalidConfigurationException | InvalidSchemaException e) { + throw new InvalidTableException(e.getMessage()); + } catch (JsonProcessingException e) { + throw new AssertionError( + "Failed to re-parse TBLPROPERTIES JSON " + tableProperties.toString()); + } + } + + private BeamTableStatistics getTableStatistics(PipelineOptions options) { + if (isBounded().equals(PCollection.IsBounded.BOUNDED)) { + return BeamTableStatistics.BOUNDED_UNKNOWN; + } + return BeamTableStatistics.UNBOUNDED_UNKNOWN; + } + + private PCollection.IsBounded isBounded() { + return getSchemaIOProvider().isBounded(); + } + + /** A generalized {@link Table} for IOs to create IO readers and writers. */ Review comment: `Table` should be `BeamSqlTable` here. `Table` is actually a class for config, I think we should rename it later.. ########## File path: sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/SchemaCapableIOTableProviderWrapper.java ########## @@ -0,0 +1,132 @@ +/* + * 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.extensions.sql.meta.provider; + +import static org.apache.beam.sdk.util.RowJsonUtils.newObjectMapperWith; + +import com.alibaba.fastjson.JSONObject; +import com.fasterxml.jackson.core.JsonProcessingException; +import java.io.Serializable; +import org.apache.beam.sdk.annotations.Experimental; +import org.apache.beam.sdk.annotations.Internal; +import org.apache.beam.sdk.extensions.sql.impl.BeamTableStatistics; +import org.apache.beam.sdk.extensions.sql.meta.BaseBeamTable; +import org.apache.beam.sdk.extensions.sql.meta.BeamSqlTable; +import org.apache.beam.sdk.extensions.sql.meta.Table; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.schemas.Schema; +import org.apache.beam.sdk.schemas.io.InvalidConfigurationException; +import org.apache.beam.sdk.schemas.io.InvalidSchemaException; +import org.apache.beam.sdk.schemas.io.SchemaIO; +import org.apache.beam.sdk.schemas.io.SchemaIOProvider; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.util.RowJson; +import org.apache.beam.sdk.values.PBegin; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.POutput; +import org.apache.beam.sdk.values.Row; + +/** + * A general {@link TableProvider} for IOs for consumption by Beam SQL. + * + * <p>Can create child classes for IOs to pass {@link #getSchemaIOProvider()} that is specific to + * the IO. + */ +@Internal +@Experimental +public abstract class SchemaCapableIOTableProviderWrapper extends InMemoryMetaTableProvider + implements Serializable { + public abstract SchemaIOProvider getSchemaIOProvider(); + + @Override + public String getTableType() { + return getSchemaIOProvider().identifier(); + } + + @Override + public BeamSqlTable buildBeamSqlTable(Table tableDefinition) { + JSONObject tableProperties = tableDefinition.getProperties(); + + try { + RowJson.RowJsonDeserializer deserializer = + RowJson.RowJsonDeserializer.forSchema(getSchemaIOProvider().configurationSchema()) + .withNullBehavior(RowJson.RowJsonDeserializer.NullBehavior.ACCEPT_MISSING_OR_NULL); + + Row configurationRow = + newObjectMapperWith(deserializer).readValue(tableProperties.toString(), Row.class); + + SchemaIO schemaIO = + getSchemaIOProvider() + .from(tableDefinition.getLocation(), configurationRow, tableDefinition.getSchema()); + + return new SchemaIOTableWrapper(schemaIO); + } catch (InvalidConfigurationException | InvalidSchemaException e) { + throw new InvalidTableException(e.getMessage()); + } catch (JsonProcessingException e) { + throw new AssertionError( + "Failed to re-parse TBLPROPERTIES JSON " + tableProperties.toString()); + } + } + + private BeamTableStatistics getTableStatistics(PipelineOptions options) { + if (isBounded().equals(PCollection.IsBounded.BOUNDED)) { + return BeamTableStatistics.BOUNDED_UNKNOWN; + } + return BeamTableStatistics.UNBOUNDED_UNKNOWN; + } + + private PCollection.IsBounded isBounded() { + return getSchemaIOProvider().isBounded(); + } + + /** A generalized {@link Table} for IOs to create IO readers and writers. */ + private class SchemaIOTableWrapper extends BaseBeamTable { + protected final SchemaIO schemaIO; + + private SchemaIOTableWrapper(SchemaIO schemaIO) { Review comment: We can make this class a static inner class by passing another parameter `PCollection.IsBounded isBounded` to the constructor and save it. This way we can also remove the 2 helper functions above (`getTableStatistics` and `isBounded`). ########## File path: sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/SchemaCapableIOTableProviderWrapper.java ########## @@ -0,0 +1,132 @@ +/* + * 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.extensions.sql.meta.provider; + +import static org.apache.beam.sdk.util.RowJsonUtils.newObjectMapperWith; + +import com.alibaba.fastjson.JSONObject; +import com.fasterxml.jackson.core.JsonProcessingException; +import java.io.Serializable; +import org.apache.beam.sdk.annotations.Experimental; +import org.apache.beam.sdk.annotations.Internal; +import org.apache.beam.sdk.extensions.sql.impl.BeamTableStatistics; +import org.apache.beam.sdk.extensions.sql.meta.BaseBeamTable; +import org.apache.beam.sdk.extensions.sql.meta.BeamSqlTable; +import org.apache.beam.sdk.extensions.sql.meta.Table; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.schemas.Schema; +import org.apache.beam.sdk.schemas.io.InvalidConfigurationException; +import org.apache.beam.sdk.schemas.io.InvalidSchemaException; +import org.apache.beam.sdk.schemas.io.SchemaIO; +import org.apache.beam.sdk.schemas.io.SchemaIOProvider; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.util.RowJson; +import org.apache.beam.sdk.values.PBegin; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.POutput; +import org.apache.beam.sdk.values.Row; + +/** + * A general {@link TableProvider} for IOs for consumption by Beam SQL. + * + * <p>Can create child classes for IOs to pass {@link #getSchemaIOProvider()} that is specific to Review comment: This line of comment belongs better as `getSchemaIOProvider()` method comment. Like: ``` // Subclasses should provide the schemaIOProvider that is specific to its IO. public abstract SchemaIOProvider getSchemaIOProvider(); ``` ########## File path: sdks/java/core/src/main/java/org/apache/beam/sdk/io/GenericRecordWriteConverter.java ########## @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.sdk.extensions.sql.meta.provider.avro; +package org.apache.beam.sdk.io; Review comment: This file is moved to the core module, I believe you want to move it to the io module? ########## File path: sdks/java/io/parquet/src/main/java/org/apache/beam/sdk/io/parquet/ParquetSchemaCapableIOProvider.java ########## @@ -0,0 +1,113 @@ +/* + * 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.parquet; + +import com.google.auto.service.AutoService; +import java.io.Serializable; +import org.apache.avro.generic.GenericRecord; +import org.apache.beam.sdk.annotations.Internal; +import org.apache.beam.sdk.schemas.Schema; +import org.apache.beam.sdk.schemas.io.SchemaIO; +import org.apache.beam.sdk.schemas.io.SchemaIOProvider; +import org.apache.beam.sdk.schemas.utils.AvroUtils; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.values.PBegin; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.POutput; +import org.apache.beam.sdk.values.Row; + +/** + * An implementation of {@link SchemaIOProvider} for reading and writing parquet files with {@link + * ParquetIO}. + */ +@Internal +@AutoService(SchemaIOProvider.class) +public class ParquetSchemaCapableIOProvider implements SchemaIOProvider { + /** Returns an id that uniquely represents this IO. */ + @Override + public String identifier() { + return "parquet"; + } + + /** + * Returns the expected schema of the configuration object. Note this is distinct from the schema + * of the data source itself. No configuration expected for parquet. + */ + @Override + public Schema configurationSchema() { + return Schema.builder().build(); + } + + /** + * Produce a SchemaIO given a String representing the data's location, the schema of the data that + * resides there, and some IO-specific configuration object. + */ + @Override + public ParquetSchemaIO from(String location, Row configuration, Schema dataSchema) { + return new ParquetSchemaIO(location, dataSchema); + } + + @Override + public boolean requiresDataSchema() { + return true; + } + + @Override + public PCollection.IsBounded isBounded() { + return PCollection.IsBounded.BOUNDED; + } + + /** An abstraction to create schema aware IOs. */ + @Internal Review comment: @Internal not necessary here. Please check in other places as well. ########## File path: sdks/java/core/src/main/java/org/apache/beam/sdk/io/AvroSchemaCapableIOProvider.java ########## @@ -0,0 +1,127 @@ +/* + * 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; + +import com.google.auto.service.AutoService; +import java.io.Serializable; +import org.apache.avro.generic.GenericRecord; +import org.apache.beam.sdk.annotations.Internal; +import org.apache.beam.sdk.schemas.Schema; +import org.apache.beam.sdk.schemas.io.SchemaIO; +import org.apache.beam.sdk.schemas.io.SchemaIOProvider; +import org.apache.beam.sdk.schemas.transforms.Convert; +import org.apache.beam.sdk.schemas.utils.AvroUtils; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.values.PBegin; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PDone; +import org.apache.beam.sdk.values.POutput; +import org.apache.beam.sdk.values.Row; + +/** + * An implementation of {@link SchemaIOProvider} for reading and writing Avro files with {@link + * AvroIO}. + */ +@Internal +@AutoService(SchemaIOProvider.class) +public class AvroSchemaCapableIOProvider implements SchemaIOProvider { + /** Returns an id that uniquely represents this IO. */ + @Override + public String identifier() { + return "avro"; + } + + /** + * Returns the expected schema of the configuration object. Note this is distinct from the schema + * of the data source itself. No configuration expected for Avro. + */ + @Override + public Schema configurationSchema() { + return Schema.builder().build(); + } + + /** + * Produce a SchemaIO given a String representing the data's location, the schema of the data that + * resides there, and some IO-specific configuration object. + */ + @Override + public AvroSchemaIO from(String location, Row configuration, Schema dataSchema) { + return new AvroSchemaIO(location, dataSchema); + } + + @Override + public boolean requiresDataSchema() { + return true; + } + + @Override + public PCollection.IsBounded isBounded() { + return PCollection.IsBounded.BOUNDED; + } + + /** An abstraction to create schema aware IOs. */ + @Internal + private static class AvroSchemaIO implements SchemaIO, Serializable { + protected final Schema dataSchema; + protected final String location; + + private AvroSchemaIO(String location, Schema dataSchema) { + this.dataSchema = dataSchema; + this.location = location; + } + + @Override + public Schema schema() { + return dataSchema; + } + + @Override + public PTransform<PBegin, PCollection<Row>> buildReader() { + return new PTransform<PBegin, PCollection<Row>>() { + @Override + public PCollection<Row> expand(PBegin begin) { + return begin + .apply( + "AvroIORead", + AvroIO.readGenericRecords(AvroUtils.toAvroSchema(dataSchema, null, null)) Review comment: This PR changes the behavior of this line, from setting the second parameter from table name to null. How will this affect the behavior of AvroIO? ########## File path: sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/parquet/ParquetTableProvider.java ########## @@ -18,13 +18,20 @@ package org.apache.beam.sdk.extensions.sql.meta.provider.parquet; import com.google.auto.service.AutoService; -import org.apache.beam.sdk.extensions.sql.meta.BeamSqlTable; -import org.apache.beam.sdk.extensions.sql.meta.Table; -import org.apache.beam.sdk.extensions.sql.meta.provider.InMemoryMetaTableProvider; +import org.apache.beam.sdk.extensions.sql.meta.provider.SchemaCapableIOTableProviderWrapper; import org.apache.beam.sdk.extensions.sql.meta.provider.TableProvider; +import org.apache.beam.sdk.io.parquet.ParquetIO; +import org.apache.beam.sdk.io.parquet.ParquetSchemaCapableIOProvider; +import org.apache.beam.sdk.schemas.io.SchemaIOProvider; /** - * {@link TableProvider} for {@link ParquetTable}. + * {@link TableProvider} for {@link ParquetIO} for consumption by Beam SQL. + * + * <p>Passes the {@link ParquetSchemaCapableIOProvider} to the generalized table provider wrapper, + * {@link SchemaCapableIOTableProviderWrapper}, for Parquet specific behavior. + * + * <p>Passes the {@link ParquetSchemaCapableIOProvider} to the generalized table provider wrapper, Review comment: Duplicate comment here. ---------------------------------------------------------------- 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. For queries about this service, please contact Infrastructure at: us...@infra.apache.org