[ https://issues.apache.org/jira/browse/BEAM-8561?focusedWorklogId=361232&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-361232 ]
ASF GitHub Bot logged work on BEAM-8561: ---------------------------------------- Author: ASF GitHub Bot Created on: 18/Dec/19 00:40 Start Date: 18/Dec/19 00:40 Worklog Time Spent: 10m Work Description: chrlarsen commented on pull request #10290: [BEAM-8561] Add ThriftIO to support IO for Thrift files URL: https://github.com/apache/beam/pull/10290#discussion_r359101649 ########## File path: sdks/java/io/thrift/src/main/java/org/apache/beam/sdk/io/thrift/ThriftIO.java ########## @@ -0,0 +1,708 @@ +/* + * 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.thrift; + +import static java.lang.String.format; +import static java.util.stream.Collectors.joining; +import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkNotNull; + +import com.google.auto.value.AutoValue; +import java.io.Closeable; +import java.io.IOException; +import java.io.OutputStream; +import java.nio.channels.Channels; +import java.nio.channels.WritableByteChannel; +import java.nio.charset.Charset; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import javax.annotation.Nullable; +import org.apache.beam.sdk.coders.StringUtf8Coder; +import org.apache.beam.sdk.io.Compression; +import org.apache.beam.sdk.io.FileIO; +import org.apache.beam.sdk.io.thrift.parser.ThriftIdlParser; +import org.apache.beam.sdk.io.thrift.parser.model.BaseType; +import org.apache.beam.sdk.io.thrift.parser.model.Const; +import org.apache.beam.sdk.io.thrift.parser.model.Definition; +import org.apache.beam.sdk.io.thrift.parser.model.Document; +import org.apache.beam.sdk.io.thrift.parser.model.Header; +import org.apache.beam.sdk.io.thrift.parser.model.IdentifierType; +import org.apache.beam.sdk.io.thrift.parser.model.IntegerEnum; +import org.apache.beam.sdk.io.thrift.parser.model.IntegerEnumField; +import org.apache.beam.sdk.io.thrift.parser.model.ListType; +import org.apache.beam.sdk.io.thrift.parser.model.MapType; +import org.apache.beam.sdk.io.thrift.parser.model.Service; +import org.apache.beam.sdk.io.thrift.parser.model.StringEnum; +import org.apache.beam.sdk.io.thrift.parser.model.Struct; +import org.apache.beam.sdk.io.thrift.parser.model.ThriftException; +import org.apache.beam.sdk.io.thrift.parser.model.ThriftField; +import org.apache.beam.sdk.io.thrift.parser.model.ThriftMethod; +import org.apache.beam.sdk.io.thrift.parser.model.ThriftType; +import org.apache.beam.sdk.io.thrift.parser.model.TypeAnnotation; +import org.apache.beam.sdk.io.thrift.parser.model.Typedef; +import org.apache.beam.sdk.io.thrift.parser.model.VoidType; +import org.apache.beam.sdk.options.ValueProvider; +import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider; +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.beam.sdk.values.PDone; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Charsets; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.io.ByteSource; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * {@link PTransform}s for reading and writing Thrift files. + * + * <h3>Reading Thrift Files</h3> + * + * <p>For simple reading, use {@link ThriftIO#read} with the desired file pattern to read from. + * + * <p>For example: + * + * <pre>{@code + * PCollection<Document> documents = pipeline.apply(ThriftIO.read().from("/foo/bar/*")); + * ... + * }</pre> + * + * <p>For more advanced use cases, like reading each file in a {@link PCollection} of {@link + * FileIO.ReadableFile}, use the {@link ReadFiles} transform. + * + * <p>For example: + * + * <pre>{@code + * PCollection<FileIO.ReadableFile> files = pipeline + * .apply(FileIO.match().filepattern(options.getInputFilepattern()) + * .apply(FileIO.readMatches()); + * + * PCollection<Document> documents = files.apply(ThriftIO.readFiles()); + * }</pre> + * + * <h3>Writing Thrift Files</h3> + * + * <p>{@link ThriftIO.Sink} allows for a {@link PCollection} of {@link Document} to be written to + * Thrift files. It can be used with the general-purpose {@link FileIO} transforms with + * FileIO.write/writeDynamic specifically. + * + * <p>By default, {@link ThriftIO.Sink} can write multiple {@link Document} to a file so it is + * highly recommended to provide a unique name for each desired file. + * + * <p>For example: + * + * <pre>{@code + * pipeline + * .apply(...) // PCollection<Document> + * .apply(FileIO + * .<Document>write() + * .via(ThriftIO.sink() + * .to("destination/path") + * .withPrefix("unique_name") + * .withSuffix(".thrift")); + * }</pre> + * + * <p>This IO API is considered experimental and may break or receive backwards-incompatible changes + * in future versions of the Apache Beam SDK. + * + * <p>NOTE: At this time retention of comments are not supported. + */ +public class ThriftIO { + + private static final String DEFAULT_THRIFT_SUFFIX = ".thrift"; + + private static final Logger LOG = LoggerFactory.getLogger(ThriftIO.class); + + /** Disable construction of utility class. */ + private ThriftIO() {} + + /** + * A {@link PTransform} that reads one or more Thrift files matching a pattern and returns a + * {@link PCollection} of {@link Document}. + */ + public static Read read() { + return new AutoValue_ThriftIO_Read.Builder().setCompression(Compression.AUTO).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_ThriftIO_ReadFiles.Builder().build(); + } + + ////////////////////////////////////////////////////////////////////////////// + + /** Implementation of {@link #read()}. */ + @AutoValue + public abstract static class Read extends PTransform<PBegin, PCollection<Document>> { + + @Nullable + abstract ValueProvider<String> getFilePattern(); + + abstract Compression getCompression(); + + abstract Builder toBuilder(); + + @AutoValue.Builder + abstract static class Builder { + abstract Builder setFilePattern(ValueProvider<String> filePattern); + + abstract Builder setCompression(Compression compression); + + abstract Read build(); + } + + /** + * Returns a transform for reading Thrift files that reads from the file(s) with the given + * filename or filename pattern. This can be a local path (if running locally), or a Google + * Cloud Storage filename or filename pattern of the form {@code "gs://<bucket>/<filepath>"} (if + * running locally or using remote execution). Standard <a + * href="http://docs.oracle.com/javase/tutorial/essential/io/find.html" >Java Filesystem glob + * patterns</a> ("*", "?", "[..]") are supported. + */ + public Read from(String filePattern) { + return from(StaticValueProvider.of(filePattern)); + } + + /** Same as {@code from(filepattern)}, but accepting a {@link ValueProvider}. */ + public Read from(ValueProvider<String> filepattern) { + return toBuilder().setFilePattern(filepattern).build(); + } + + /** + * Returns a transform for reading Thrift files that decompresses all input files using the + * specified compression type. + * + * <p>If no compression type is specified, the default is {@link Compression#AUTO}. In this + * mode, the compression type of the file is determined by it's extension via {@link + * Compression#detect(String)}. + */ + public Read withCompression(Compression compression) { + return toBuilder().setCompression(compression).build(); + } + + @Override + public PCollection<Document> expand(PBegin input) { + + return input + .apply("Create filepattern", Create.ofProvider(getFilePattern(), StringUtf8Coder.of())) Review comment: Yes I've verified this works with Chinese and Armenian characters in the file pattern. ---------------------------------------------------------------- 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 Issue Time Tracking ------------------- Worklog Id: (was: 361232) Time Spent: 7h 50m (was: 7h 40m) > Add ThriftIO to Support IO for Thrift Files > ------------------------------------------- > > Key: BEAM-8561 > URL: https://issues.apache.org/jira/browse/BEAM-8561 > Project: Beam > Issue Type: New Feature > Components: io-java-files > Reporter: Chris Larsen > Assignee: Chris Larsen > Priority: Major > Time Spent: 7h 50m > Remaining Estimate: 0h > > Similar to AvroIO it would be very useful to support reading and writing > to/from Thrift files with a native connector. > Functionality would include: > # read() - Reading from one or more Thrift files. > # write() - Writing to one or more Thrift files. -- This message was sent by Atlassian Jira (v8.3.4#803005)