iemejia commented on a change in pull request #14117:
URL: https://github.com/apache/beam/pull/14117#discussion_r585448320



##########
File path: 
sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/parquet/ParquetTable.java
##########
@@ -0,0 +1,119 @@
+/*
+ * 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.parquet;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import org.apache.avro.Schema;
+import org.apache.avro.Schema.Field;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.beam.sdk.annotations.Internal;
+import org.apache.beam.sdk.extensions.sql.impl.BeamTableStatistics;
+import org.apache.beam.sdk.extensions.sql.meta.BeamSqlTableFilter;
+import org.apache.beam.sdk.extensions.sql.meta.ProjectSupport;
+import org.apache.beam.sdk.extensions.sql.meta.SchemaBaseBeamTable;
+import org.apache.beam.sdk.extensions.sql.meta.Table;
+import org.apache.beam.sdk.io.FileIO;
+import org.apache.beam.sdk.io.parquet.ParquetIO;
+import org.apache.beam.sdk.io.parquet.ParquetIO.Read;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.schemas.transforms.Convert;
+import org.apache.beam.sdk.schemas.utils.AvroUtils;
+import org.apache.beam.sdk.values.PBegin;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollection.IsBounded;
+import org.apache.beam.sdk.values.POutput;
+import org.apache.beam.sdk.values.Row;
+
+@Internal
+@SuppressWarnings({"nullness"})
+class ParquetTable extends SchemaBaseBeamTable implements Serializable {
+  private final Table table;
+
+  ParquetTable(Table table) {
+    super(table.getSchema());
+    this.table = table;
+  }
+
+  @Override
+  public PCollection<Row> buildIOReader(PBegin begin) {
+    final Schema schema = AvroUtils.toAvroSchema(table.getSchema());
+    Read read = 
ParquetIO.read(schema).withBeamSchemas(true).from(table.getLocation() + "/*");
+    return begin.apply("ParquetIORead", read).apply("ToRows", 
Convert.toRows());
+  }
+
+  @Override
+  public PCollection<Row> buildIOReader(
+      PBegin begin, BeamSqlTableFilter filters, List<String> fieldNames) {
+    final Schema schema = AvroUtils.toAvroSchema(table.getSchema());
+    Read read = 
ParquetIO.read(schema).withBeamSchemas(true).from(table.getLocation() + "/*");
+    if (!fieldNames.isEmpty()) {
+      Schema projectionSchema = projectSchema(schema, fieldNames);
+      read = read.withProjection(projectionSchema, projectionSchema);

Review comment:
       Done. I won't do the TODO since this is probably my next PR 
[BEAM-11908](https://issues.apache.org/jira/browse/BEAM-11908) Deprecate 
.withProjection from ParquetIO
   However I have a doubt I agre with you that the list of Strings (order 
matters) is the nicest possible API but historically on Beam we try to avoid 
redoing the APIs of the data sources, so I am wondering if we should not better 
just let users do that in the HadoopConfiguration object as they do in upstream 
Parquet. More details in the ticket.
   Tell me what you think and I will take care of (1) future removal or (2) 
extra API withProjection(conf, fields);

##########
File path: sdks/java/extensions/sql/build.gradle
##########
@@ -79,6 +79,7 @@ dependencies {
   provided project(":sdks:java:io:kafka")
   provided project(":sdks:java:io:google-cloud-platform")
   compile project(":sdks:java:io:mongodb")
+  compile library.java.avro

Review comment:
       I was obliged to do this because of the strict dependency analysis 
enabled now forces to make use of dependencies explicitly in every module, but 
Avro in particular should not matter much remember that we leak Avro from 
`sdks/java/core` so this should not represent any issue.

##########
File path: 
sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/parquet/ParquetTable.java
##########
@@ -0,0 +1,119 @@
+/*
+ * 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.parquet;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import org.apache.avro.Schema;
+import org.apache.avro.Schema.Field;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.beam.sdk.annotations.Internal;
+import org.apache.beam.sdk.extensions.sql.impl.BeamTableStatistics;
+import org.apache.beam.sdk.extensions.sql.meta.BeamSqlTableFilter;
+import org.apache.beam.sdk.extensions.sql.meta.ProjectSupport;
+import org.apache.beam.sdk.extensions.sql.meta.SchemaBaseBeamTable;
+import org.apache.beam.sdk.extensions.sql.meta.Table;
+import org.apache.beam.sdk.io.FileIO;
+import org.apache.beam.sdk.io.parquet.ParquetIO;
+import org.apache.beam.sdk.io.parquet.ParquetIO.Read;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.schemas.transforms.Convert;
+import org.apache.beam.sdk.schemas.utils.AvroUtils;
+import org.apache.beam.sdk.values.PBegin;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollection.IsBounded;
+import org.apache.beam.sdk.values.POutput;
+import org.apache.beam.sdk.values.Row;
+
+@Internal
+@SuppressWarnings({"nullness"})
+class ParquetTable extends SchemaBaseBeamTable implements Serializable {
+  private final Table table;
+
+  ParquetTable(Table table) {
+    super(table.getSchema());
+    this.table = table;
+  }
+
+  @Override
+  public PCollection<Row> buildIOReader(PBegin begin) {
+    final Schema schema = AvroUtils.toAvroSchema(table.getSchema());
+    Read read = 
ParquetIO.read(schema).withBeamSchemas(true).from(table.getLocation() + "/*");
+    return begin.apply("ParquetIORead", read).apply("ToRows", 
Convert.toRows());
+  }
+
+  @Override
+  public PCollection<Row> buildIOReader(
+      PBegin begin, BeamSqlTableFilter filters, List<String> fieldNames) {
+    final Schema schema = AvroUtils.toAvroSchema(table.getSchema());
+    Read read = 
ParquetIO.read(schema).withBeamSchemas(true).from(table.getLocation() + "/*");
+    if (!fieldNames.isEmpty()) {
+      Schema projectionSchema = projectSchema(schema, fieldNames);
+      read = read.withProjection(projectionSchema, projectionSchema);

Review comment:
       Done. I won't do the TODO since this is probably my next PR [BEAM-11908 
Deprecate .withProjection from 
ParquetIO](https://issues.apache.org/jira/browse/BEAM-11908). 
   I agree with about the list of Strings (order matters) as the nicest 
possible API but historically on Beam we try to avoid redoing the APIs of the 
data sources that users can do externally to the IO, so I am wondering if we 
should not better just let users do that in the HadoopConfiguration object as 
they do in upstream Parquet/Spark RDD. More details in the ticket.
   
   Tell me what you think and I will take care of (1) future removal + extra 
javadoc or (2) extra API withProjection(conf, fields);

##########
File path: 
sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/parquet/ParquetTableProvider.java
##########
@@ -39,19 +36,18 @@
  *   favorite_numbers ARRAY<INTEGER>
  * )
  * TYPE 'parquet'
- * LOCATION '/home/admin/users.parquet'
+ * LOCATION '/home/admin/orders/'

Review comment:
       Agree this should be up to the user to specify exactly but there is an 
issue, you need a 'directory'  for the write so maybe we should specify this 
via some property like [they discuss here for 
Hive](https://stackoverflow.com/questions/11269203/when-creating-an-external-table-in-hive-can-i-point-the-location-to-specific-fil)?
 or do you have other idea(s)?
   




----------------------------------------------------------------
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:
[email protected]


Reply via email to