From commits-return-72051-archive-asf-public=cust-asf.ponee.io@beam.apache.org Fri May 11 22:17:06 2018 Return-Path: X-Original-To: archive-asf-public@cust-asf.ponee.io Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by mx-eu-01.ponee.io (Postfix) with SMTP id E2AE9180647 for ; Fri, 11 May 2018 22:17:05 +0200 (CEST) Received: (qmail 20484 invoked by uid 500); 11 May 2018 20:17:05 -0000 Mailing-List: contact commits-help@beam.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@beam.apache.org Delivered-To: mailing list commits@beam.apache.org Received: (qmail 20465 invoked by uid 99); 11 May 2018 20:17:04 -0000 Received: from pnap-us-west-generic-nat.apache.org (HELO spamd1-us-west.apache.org) (209.188.14.142) by apache.org (qpsmtpd/0.29) with ESMTP; Fri, 11 May 2018 20:17:04 +0000 Received: from localhost (localhost [127.0.0.1]) by spamd1-us-west.apache.org (ASF Mail Server at spamd1-us-west.apache.org) with ESMTP id 8C1F7C1B2B for ; Fri, 11 May 2018 20:17:04 +0000 (UTC) X-Virus-Scanned: Debian amavisd-new at spamd1-us-west.apache.org X-Spam-Flag: NO X-Spam-Score: -109.5 X-Spam-Level: X-Spam-Status: No, score=-109.5 tagged_above=-999 required=6.31 tests=[ENV_AND_HDR_SPF_MATCH=-0.5, KAM_ASCII_DIVIDERS=0.8, RCVD_IN_DNSWL_MED=-2.3, SPF_PASS=-0.001, URIBL_BLOCKED=0.001, USER_IN_DEF_SPF_WL=-7.5, USER_IN_WHITELIST=-100] autolearn=disabled Received: from mx1-lw-us.apache.org ([10.40.0.8]) by localhost (spamd1-us-west.apache.org [10.40.0.7]) (amavisd-new, port 10024) with ESMTP id Za8ehTe1nI4E for ; Fri, 11 May 2018 20:17:03 +0000 (UTC) Received: from mailrelay1-us-west.apache.org (mailrelay1-us-west.apache.org [209.188.14.139]) by mx1-lw-us.apache.org (ASF Mail Server at mx1-lw-us.apache.org) with ESMTP id C73125FAD2 for ; Fri, 11 May 2018 20:17:02 +0000 (UTC) Received: from jira-lw-us.apache.org (unknown [207.244.88.139]) by mailrelay1-us-west.apache.org (ASF Mail Server at mailrelay1-us-west.apache.org) with ESMTP id 8C6CFE0F96 for ; Fri, 11 May 2018 20:17:01 +0000 (UTC) Received: from jira-lw-us.apache.org (localhost [127.0.0.1]) by jira-lw-us.apache.org (ASF Mail Server at jira-lw-us.apache.org) with ESMTP id A41E82154D for ; Fri, 11 May 2018 20:17:00 +0000 (UTC) Date: Fri, 11 May 2018 20:17:00 +0000 (UTC) From: "ASF GitHub Bot (JIRA)" To: commits@beam.apache.org Message-ID: In-Reply-To: References: Subject: [jira] [Work logged] (BEAM-214) Create Parquet IO MIME-Version: 1.0 Content-Type: text/plain; charset=utf-8 Content-Transfer-Encoding: quoted-printable X-JIRA-FingerPrint: 30527f35849b9dde25b450d4833f0394 [ https://issues.apache.org/jira/browse/BEAM-214?focusedWorklogId=3D10= 1292&page=3Dcom.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpane= l#worklog-101292 ] ASF GitHub Bot logged work on BEAM-214: --------------------------------------- Author: ASF GitHub Bot Created on: 11/May/18 20:16 Start Date: 11/May/18 20:16 Worklog Time Spent: 10m=20 Work Description: jkff commented on a change in pull request #5242: [= BEAM-214] ParquetIO URL: https://github.com/apache/beam/pull/5242#discussion_r187721225 =20 =20 ########## File path: sdks/java/io/parquet/src/main/java/org/apache/beam/sdk/io/parqu= et/ParquetIO.java ########## @@ -0,0 +1,562 @@ +/* + * 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 static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.base.Preconditions.checkNotNull; +import static org.apache.parquet.hadoop.ParquetFileWriter.Mode.OVERWRITE; + +import com.google.auto.value.AutoValue; +import java.io.ByteArrayInputStream; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.nio.channels.Channels; +import java.nio.channels.SeekableByteChannel; +import java.nio.channels.WritableByteChannel; +import javax.annotation.Nullable; +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericRecord; +import org.apache.beam.sdk.annotations.Experimental; +import org.apache.beam.sdk.coders.AvroCoder; +import org.apache.beam.sdk.coders.StringUtf8Coder; +import org.apache.beam.sdk.io.FileIO; +import org.apache.beam.sdk.io.FileSystems; +import org.apache.beam.sdk.io.fs.EmptyMatchTreatment; +import org.apache.beam.sdk.io.fs.MatchResult; +import org.apache.beam.sdk.io.fs.ResourceId; +import org.apache.beam.sdk.options.ValueProvider; +import org.apache.beam.sdk.transforms.Create; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.transforms.Watch; +import org.apache.beam.sdk.transforms.display.DisplayData; +import org.apache.beam.sdk.values.PBegin; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PDone; +import org.apache.parquet.avro.AvroParquetReader; +import org.apache.parquet.avro.AvroParquetWriter; +import org.apache.parquet.hadoop.ParquetReader; +import org.apache.parquet.hadoop.ParquetWriter; +import org.apache.parquet.io.DelegatingSeekableInputStream; +import org.apache.parquet.io.InputFile; +import org.apache.parquet.io.OutputFile; +import org.apache.parquet.io.PositionOutputStream; +import org.apache.parquet.io.SeekableInputStream; +import org.joda.time.Duration; + +/** + * IO to read and write Parquet files. + * + *

Reading Parquet files

+ * + *

{@link ParquetIO} source returns a {@link PCollection} for + * Parquet files. The elements in the {@link PCollection} are Avro {@link = GenericRecord}. + * + *

To configure the {@link Read}, you have to provide the file patterns= (from) of the + * Parquet files and the Avro schema. + * + *

For example: + * + *

{@code
+ *  pipeline.apply(ParquetIO.read().from("/foo/bar").withSchema(schema))
+ *  ...
+ * }
+ * 
+ * + *

As {@link Read} is based on {@link FileIO}, it supports any filesyst= em (hdfs, ...). + * + *

Writing Parquet files

+ * + *

{@link Write} allows you to write a {@link PCollection} of {@link Ge= nericRecord} into a + * Parquet file. + * + *

For example: + * + *

{@code
+ *  pipeline
+ *    .apply(...) // PCollection
+ *    .apply(ParquetIO.write().to("/foo/bar").withSchema(schema));
+ * }
+ */ +public class ParquetIO { + + /** + * Reads {@link GenericRecord} from a Parquet file (or multiple Parquet = files matching + * the pattern). + */ + public static Read read() { + return new AutoValue_ParquetIO_Read.Builder().setHintMatchesManyFiles(= false) + .setMatchConfiguration(FileIO.MatchConfiguration.create(EmptyMatch= Treatment.DISALLOW)) + .build(); + } + + /** + * Like {@link #read()}, but reads each filepattern in the input {@link = PCollection}. + */ + public static ReadAll readAll() { + return new AutoValue_ParquetIO_ReadAll.Builder() + .setMatchConfiguration(FileIO.MatchConfiguration + .create(EmptyMatchTreatment.ALLOW_IF_WILDCARD)) + .build(); + } + + /** + * Like {@link #read()}, but reads each file in a {@link PCollection} + * of {@link org.apache.beam.sdk.io.FileIO.ReadableFile}, + * which allows more flexible usage. + */ + public static ReadFiles readFiles() { + return new AutoValue_ParquetIO_ReadFiles.Builder().build(); + } + + /** + * Writes a {@link PCollection} to an Parquet file. + */ + public static Write write() { + return new AutoValue_ParquetIO_Write.Builder().build(); + } + + /** + * Implementation of {@link #read()}. + */ + @AutoValue + public abstract static class Read extends PTransform> { + + @Nullable abstract ValueProvider filepattern(); + abstract FileIO.MatchConfiguration matchConfiguration(); + @Nullable abstract Schema schema(); + abstract boolean hintMatchesManyFiles(); + + abstract Builder builder(); + + @AutoValue.Builder + abstract static class Builder { + abstract Builder setFilepattern(ValueProvider filepattern); + abstract Builder setMatchConfiguration(FileIO.MatchConfiguration mat= chConfiguration); + abstract Builder setSchema(Schema schema); + abstract Builder setHintMatchesManyFiles(boolean hintManyFiles); + + abstract Read build(); + } + + /** + * Reads from the given filename or filepattern. + * + *

If it is known that the filepattern will match a very large numb= er of files (at least tens + * of thousands), use {@link #withHintMatchesManyFiles} for better per= formance and scalability. + */ + public Read from(ValueProvider filepattern) { + return builder().setFilepattern(filepattern).build(); + } + + /** Like {@link #from(ValueProvider)}. */ + public Read from(String filepattern) { + return from(ValueProvider.StaticValueProvider.of(filepattern)); + } + + /** + * Schema of the record in the Parquet file. + */ + public Read withSchema(Schema schema) { + return builder().setSchema(schema).build(); + } + + /** Sets the {@link FileIO.MatchConfiguration}. */ + public Read withMatchConfiguration(FileIO.MatchConfiguration matchConf= iguration) { + return builder().setMatchConfiguration(matchConfiguration).build(); + } + + /** Configures whether or not a filepattern matching no files is allow= ed. */ + public Read withEmptyMatchTreatment(EmptyMatchTreatment treatment) { + return withMatchConfiguration(matchConfiguration().withEmptyMatchTre= atment(treatment)); + } + + /** + * Continuously watches for new files matching the filepattern, pollin= g it at the given + * interval, until the given termination condition is reached. The ret= urned {@link PCollection} + * is unbounded. + * + *

This works only in runners supporting {@link Experimental.Kind#S= PLITTABLE_DO_FN}. + */ + @Experimental(Experimental.Kind.SPLITTABLE_DO_FN) + public Read watchForNewFiles( + Duration pollInterval, Watch.Growth.TerminationCondition terminationCondition) { + return withMatchConfiguration( + matchConfiguration().continuously(pollInterval, terminationCondi= tion)); + } + + /** + * Hints that the filepattern specified in {@link #from(String)} match= es a very large number of + * files. + * + *

This hint may cause a runner to execute the transform differentl= y, in a way that improves + * performance for this case, but it may worsen performance if the fil= epattern matches only a + * small number of files (e.g., in a runner that supports dynamic work= rebalancing, it will + * happen less efficiently within individual files). + */ + public Read withHintMatchesManyFiles() { + return builder().setHintMatchesManyFiles(true).build(); + } + + @Override + public PCollection expand(PBegin input) { + checkNotNull(filepattern(), "filepattern"); + checkNotNull(schema(), "schema"); + + ReadAll readAll =3D readAll().withMatchConfiguration(matchConfigurat= ion()).withSchema(schema()); + return input + .apply("Create filepattern", Create.ofProvider(filepattern(), + StringUtf8Coder.of())) + .apply("Via ReadAll", readAll); + } + + @Override + public void populateDisplayData(DisplayData.Builder builder) { + super.populateDisplayData(builder); + builder + .addIfNotNull( + DisplayData.item("filePattern", filepattern()).withLabel("In= put File Pattern")) + .include("matchConfiguration", matchConfiguration()); + } + } + + /** + * Implementation of {@link #readAll()}. + */ + @AutoValue + public abstract static class ReadAll extends PTransform, + PCollection> { + + abstract FileIO.MatchConfiguration getMatchConfiguration(); + @Nullable abstract Schema getSchema(); + + abstract Builder builder(); + + @AutoValue.Builder + abstract static class Builder { + abstract Builder setMatchConfiguration(FileIO.MatchConfiguration mat= chConfiguration); + abstract Builder setSchema(Schema schema); + + abstract ReadAll build(); + } + + /** + * Sets the {@link org.apache.beam.sdk.io.FileIO.MatchConfiguration}. + */ + public ReadAll withMatchConfiguration(FileIO.MatchConfiguration config= uration) { + return builder().setMatchConfiguration(configuration).build(); + } + + /** + * Sets the {@link EmptyMatchTreatment}. + */ + public ReadAll withEmptyMatchTreatment(EmptyMatchTreatment treatment) = { + return withMatchConfiguration(getMatchConfiguration().withEmptyMatch= Treatment(treatment)); + } + + /** + * Sets the schema of the records. + */ + public ReadAll withSchema(Schema schema) { + return builder().setSchema(schema).build(); + } + + @Experimental(Experimental.Kind.SPLITTABLE_DO_FN) + public ReadAll watchForNewFiles( + Duration pollInterval, Watch.Growth.TerminationCondition terminationCondition) { + return withMatchConfiguration( + getMatchConfiguration().continuously(pollInterval, terminationCo= ndition)); + } + + @Override + public PCollection expand(PCollection input) { + checkNotNull(getSchema(), "schema"); + return input + .apply(FileIO.matchAll().withConfiguration(getMatchConfiguration= ())) + .apply(FileIO.readMatches()) + .apply(readFiles().withSchema(getSchema())); + } + + @Override + public void populateDisplayData(DisplayData.Builder builder) { + super.populateDisplayData(builder); + builder.include("matchConfiguration", getMatchConfiguration()); + } + + } + + /** + * Implementation of {@link #readFiles()}. + */ + @AutoValue + public abstract static class ReadFiles extends PTransform, + PCollection> { + + @Nullable abstract Schema schema(); + + abstract Builder builder(); + + @AutoValue.Builder + abstract static class Builder { + abstract Builder setSchema(Schema schema); + abstract ReadFiles build(); + } + + /** + * Define the Avro schema of the record to read from the Parquet file. + */ + public ReadFiles withSchema(Schema schema) { + checkArgument(schema !=3D null, + "schema can not be null"); + return builder().setSchema(schema).build(); + } + + @Override + public PCollection expand(PCollection input) { + return input + .apply(ParDo.of(new ReadFn())) =20 Review comment: This means the reading of Parquet files will not be splittable, and it w= ill be particularly problematic in some streaming runners eg. Dataflow. It = may be fine for a first version, but is there a relatively easy way to make= it splittable? ---------------------------------------------------------------- This is an automated message from the Apache Git Service. To respond to the message, please log on GitHub and use the URL above to go to the specific comment. =20 For queries about this service, please contact Infrastructure at: users@infra.apache.org Issue Time Tracking ------------------- Worklog Id: (was: 101292) Time Spent: 7h 10m (was: 7h) > Create Parquet IO > ----------------- > > Key: BEAM-214 > URL: https://issues.apache.org/jira/browse/BEAM-214 > Project: Beam > Issue Type: Improvement > Components: io-ideas > Reporter: Neville Li > Assignee: Jean-Baptiste Onofr=C3=A9 > Priority: Minor > Time Spent: 7h 10m > Remaining Estimate: 0h > > Would be nice to support Parquet files with projection and predicates. -- This message was sent by Atlassian JIRA (v7.6.3#76005)