From commits-return-73933-archive-asf-public=cust-asf.ponee.io@beam.apache.org Tue May 22 20:30:05 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 AB0A518067E for ; Tue, 22 May 2018 20:30:04 +0200 (CEST) Received: (qmail 83960 invoked by uid 500); 22 May 2018 18:30:03 -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 83840 invoked by uid 99); 22 May 2018 18:30:03 -0000 Received: from pnap-us-west-generic-nat.apache.org (HELO spamd4-us-west.apache.org) (209.188.14.142) by apache.org (qpsmtpd/0.29) with ESMTP; Tue, 22 May 2018 18:30:03 +0000 Received: from localhost (localhost [127.0.0.1]) by spamd4-us-west.apache.org (ASF Mail Server at spamd4-us-west.apache.org) with ESMTP id 63064C012D for ; Tue, 22 May 2018 18:30:03 +0000 (UTC) X-Virus-Scanned: Debian amavisd-new at spamd4-us-west.apache.org X-Spam-Flag: NO X-Spam-Score: -109.511 X-Spam-Level: X-Spam-Status: No, score=-109.511 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, T_RP_MATCHES_RCVD=-0.01, 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 (spamd4-us-west.apache.org [10.40.0.11]) (amavisd-new, port 10024) with ESMTP id Vgto0Ob8Lz-x for ; Tue, 22 May 2018 18:30:01 +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 1FBC95F5B2 for ; Tue, 22 May 2018 18:30:01 +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 8F9FDE0C50 for ; Tue, 22 May 2018 18:30:00 +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 45E9B217E0 for ; Tue, 22 May 2018 18:30:00 +0000 (UTC) Date: Tue, 22 May 2018 18:30: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= 4745&page=3Dcom.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpane= l#worklog-104745 ] ASF GitHub Bot logged work on BEAM-214: --------------------------------------- Author: ASF GitHub Bot Created on: 22/May/18 18:29 Start Date: 22/May/18 18:29 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_r190006475 =20 =20 ########## File path: sdks/java/io/parquet/src/main/java/org/apache/beam/sdk/io/parqu= et/ParquetIO.java ########## @@ -0,0 +1,371 @@ +/* + * 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.checkNotNull; +import static org.apache.parquet.hadoop.ParquetFileWriter.Mode.OVERWRITE; + +import com.google.auto.value.AutoValue; +import java.io.IOException; +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.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.display.DisplayData; +import org.apache.beam.sdk.values.PBegin; +import org.apache.beam.sdk.values.PCollection; +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; + +/** + * 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 coder. + * + *

For example: + * + *

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

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

Writing Parquet files

+ * + *

{@link ParquetIO.Sink} allows you to write a {@link PCollection} of = {@link GenericRecord} + * into a Parquet file. It can be used with the general-purpose {@link Fil= eIO} transforms. + * + *

For example: + * + *

{@code
+ *  pipeline
+ *    .apply(...) // PCollection
+ *    .apply(FileIO.
+ *      .write()
+ *      .via(ParquetIO.sink(SCHEMA))
+ *      .to("destination/path")
+ * }
+ * + *

This IO API is considered experimental and may break or receive + * backwards-incompatible changes in future versions of the Apache Beam SD= K. + */ +@Experimental(Experimental.Kind.SOURCE_SINK) +public class ParquetIO { + + /** + * Reads {@link GenericRecord} from a Parquet file (or multiple Parquet = files matching + * the pattern). + */ + public static Read read(Schema schema) { + return new AutoValue_ParquetIO_Read.Builder() + .setSchema(schema) + .build(); + } + + /** + * Like {@link #read(Schema)}, but reads each file in a {@link PCollecti= on} + * of {@link org.apache.beam.sdk.io.FileIO.ReadableFile}, + * which allows more flexible usage. + */ + public static ReadFiles readFiles(Schema schema) { + return new AutoValue_ParquetIO_ReadFiles.Builder() + .setSchema(schema) + .build(); + } + + /** + * Implementation of {@link #read(Schema)}. + */ + @AutoValue + public abstract static class Read extends PTransform> { + + @Nullable + abstract ValueProvider getFilepattern(); + + @Nullable + abstract Schema getSchema(); + + abstract Builder builder(); + + @AutoValue.Builder + abstract static class Builder { + abstract Builder setFilepattern(ValueProvider filepattern); + abstract Builder setSchema(Schema schema); + + abstract Read build(); + } + + /** + * Reads from the given filename or filepattern. + */ + 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)); + } + + @Override + public PCollection expand(PBegin input) { + checkNotNull(getFilepattern(), "Filepattern cannot be null."); + + return input + .apply("Create filepattern", Create.ofProvider(getFilepattern(), S= tringUtf8Coder.of())) + .apply(FileIO.matchAll()) + .apply(FileIO.readMatches()) + .apply(readFiles(getSchema())); + } + + @Override + public void populateDisplayData(DisplayData.Builder builder) { + super.populateDisplayData(builder); + builder + .add(DisplayData.item("filePattern", getFilepattern()).withLabel("= Input File Pattern")); + } + } + + /** + * Implementation of {@link #readFiles(Schema)}. + */ + @AutoValue + public abstract static class ReadFiles extends PTransform, + PCollection> { + + @Nullable + abstract Schema getSchema(); + + @AutoValue.Builder + abstract static class Builder { + abstract Builder setSchema(Schema schema); + abstract ReadFiles build(); + } + + @Override + public PCollection expand(PCollection input) { + checkNotNull(getSchema(), "Schema can not be null"); + return input.apply(ParDo.of(new ReadFn())).setCoder(AvroCoder.of(get= Schema())); + } + + static class ReadFn extends DoFn { + + @ProcessElement + public void processElement(ProcessContext processContext) throws Exc= eption { + FileIO.ReadableFile file =3D processContext.element(); + + if (!file.getMetadata().isReadSeekEfficient()) { + String filename =3D file.getMetadata().resourceId().getFilename(= ); + throw new RuntimeException(String.format("File has to be seekabl= e: %s", filename)); =20 Review comment: I think "getFilename()" returns only the last path component. Just use r= esourceId() itself as the %s. ---------------------------------------------------------------- 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: 104745) Time Spent: 13h 10m (was: 13h) > 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: 13h 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)