You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@beam.apache.org by "ASF GitHub Bot (Jira)" <ji...@apache.org> on 2021/03/02 00:04:00 UTC

[jira] [Work logged] (BEAM-7929) ParquetTable.buildIOReader should support column projection and filter predicate

     [ https://issues.apache.org/jira/browse/BEAM-7929?focusedWorklogId=559650&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-559650 ]

ASF GitHub Bot logged work on BEAM-7929:
----------------------------------------

                Author: ASF GitHub Bot
            Created on: 02/Mar/21 00:03
            Start Date: 02/Mar/21 00:03
    Worklog Time Spent: 10m 
      Work Description: TheNeuralBit commented on a change in pull request #14117:
URL: https://github.com/apache/beam/pull/14117#discussion_r585142707



##########
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:
       Yeah this is strange.. unfortunately it looks like it's been part of the public API for a few releases. Shouldn't be too difficult to deprecate and keep the backwards compatible version though.
   
   I feel like this API should just accept a collection of Strings representing column names, we should already have the type information right?
   
   Based on these docs I'm not sure what you have will work, it seems like the two schemas should be different: https://github.com/apache/beam/blob/master/sdks/java/io/parquet/src/main/java/org/apache/beam/sdk/io/parquet/ParquetIO.java#L150-L152




----------------------------------------------------------------
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:
users@infra.apache.org


Issue Time Tracking
-------------------

    Worklog Id:     (was: 559650)
    Time Spent: 0.5h  (was: 20m)

> ParquetTable.buildIOReader should support column projection and filter predicate
> --------------------------------------------------------------------------------
>
>                 Key: BEAM-7929
>                 URL: https://issues.apache.org/jira/browse/BEAM-7929
>             Project: Beam
>          Issue Type: New Feature
>          Components: dsl-sql, io-java-parquet
>    Affects Versions: 2.14.0
>            Reporter: Neville Li
>            Priority: P3
>          Time Spent: 0.5h
>  Remaining Estimate: 0h
>
> To leverage the performance improvements in Parquet.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)