Skip to content

Commit

Permalink
[FLINK-9753] [formats] Add a Parquet BulkWriter
Browse files Browse the repository at this point in the history
  • Loading branch information
StephanEwen committed Jul 20, 2018
1 parent b56c75c commit 66b1f85
Show file tree
Hide file tree
Showing 16 changed files with 1,450 additions and 0 deletions.
160 changes: 160 additions & 0 deletions flink-formats/flink-parquet/pom.xml
Original file line number Diff line number Diff line change
@@ -0,0 +1,160 @@
<?xml version="1.0" encoding="UTF-8"?>
<!--
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
https://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.
-->
<project xmlns="https://maven.apache.org/POM/4.0.0" xmlns:xsi="https://www.w3.org/2001/XMLSchema-instance"
xsi:schemaLocation="https://maven.apache.org/POM/4.0.0 https://maven.apache.org/maven-v4_0_0.xsd">

<modelVersion>4.0.0</modelVersion>

<parent>
<groupId>org.apache.flink</groupId>
<artifactId>flink-formats</artifactId>
<version>1.7-SNAPSHOT</version>
<relativePath>..</relativePath>
</parent>

<artifactId>flink-parquet</artifactId>
<name>flink-parquet</name>

<packaging>jar</packaging>

<properties>
<flink.format.parquet.version>1.10.0</flink.format.parquet.version>
</properties>

<dependencies>

<!-- Flink dependencies -->

<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-core</artifactId>
<version>${project.version}</version>
<scope>provided</scope>
</dependency>

<!-- Parquet Dependencies -->

<dependency>
<groupId>org.apache.parquet</groupId>
<artifactId>parquet-hadoop</artifactId>
<version>${flink.format.parquet.version}</version>
</dependency>

<!-- Hadoop is needed by Parquet -->
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-shaded-hadoop2</artifactId>
<version>${project.version}</version>
<scope>provided</scope>
</dependency>

<!-- For now, fastutil is provided already by flink-runtime -->
<dependency>
<groupId>it.unimi.dsi</groupId>
<artifactId>fastutil</artifactId>
<version>8.2.1</version>
<scope>provided</scope>
</dependency>

<!-- Optional Parquet Builders for Formats like Avro, Protobuf, Thrift -->

<dependency>
<groupId>org.apache.parquet</groupId>
<artifactId>parquet-avro</artifactId>
<version>${flink.format.parquet.version}</version>
<optional>true</optional>
<exclusions>
<exclusion>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-client</artifactId>
</exclusion>
<exclusion>
<groupId>it.unimi.dsi</groupId>
<artifactId>fastutil</artifactId>
</exclusion>
</exclusions>
</dependency>

<!-- test dependencies -->

<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-test-utils_2.11</artifactId>
<version>${project.version}</version>
<scope>test</scope>
</dependency>

<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-streaming-java_2.11</artifactId>
<version>${project.version}</version>
<scope>test</scope>
</dependency>

<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-avro</artifactId>
<version>${project.version}</version>
<scope>test</scope>
</dependency>

</dependencies>


<build>
<plugins>
<!-- Generate Test class from avro schema -->
<plugin>
<groupId>org.apache.avro</groupId>
<artifactId>avro-maven-plugin</artifactId>
<version>${avro.version}</version>
<executions>
<execution>
<phase>generate-sources</phase>
<goals>
<goal>schema</goal>
</goals>
<configuration>
<testSourceDirectory>${project.basedir}/src/test/resources/avro</testSourceDirectory>
<testOutputDirectory>${project.basedir}/src/test/java/</testOutputDirectory>
</configuration>
</execution>
</executions>
</plugin>

<!-- skip dependency convergence due to Hadoop dependency -->
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-enforcer-plugin</artifactId>
<executions>
<execution>
<id>dependency-convergence</id>
<goals>
<goal>enforce</goal>
</goals>
<configuration>
<skip>true</skip>
</configuration>
</execution>
</executions>
</plugin>
</plugins>
</build>
</project>
Original file line number Diff line number Diff line change
@@ -0,0 +1,39 @@
/*
* 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
*
* https://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.flink.formats.parquet;

import org.apache.parquet.hadoop.ParquetWriter;
import org.apache.parquet.io.OutputFile;

import java.io.IOException;
import java.io.Serializable;

/**
* A builder to create a {@link ParquetWriter} from a Parquet {@link OutputFile}.
*
* @param <T> The type of elements written by the writer.
*/
@FunctionalInterface
public interface ParquetBuilder<T> extends Serializable {

/**
* Creates and configures a parquet writer to the given output file.
*/
ParquetWriter<T> createWriter(OutputFile out) throws IOException;
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,64 @@
/*
* 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
*
* https://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.flink.formats.parquet;

import org.apache.flink.annotation.PublicEvolving;
import org.apache.flink.api.common.serialization.BulkWriter;

import org.apache.parquet.hadoop.ParquetWriter;

import java.io.IOException;

import static org.apache.flink.util.Preconditions.checkNotNull;

/**
* A simple {@link BulkWriter} implementation that wraps a {@link ParquetWriter}.
*
* @param <T> The type of records written.
*/
@PublicEvolving
public class ParquetBulkWriter<T> implements BulkWriter<T> {

/** The ParquetWriter to write to. */
private final ParquetWriter<T> parquetWriter;

/**
* Creates a new ParquetBulkWriter wrapping the given ParquetWriter.
*
* @param parquetWriter The ParquetWriter to write to.
*/
public ParquetBulkWriter(ParquetWriter<T> parquetWriter) {
this.parquetWriter = checkNotNull(parquetWriter, "parquetWriter");
}

@Override
public void addElement(T datum) throws IOException {
parquetWriter.write(datum);
}

@Override
public void flush() {
// nothing we can do here
}

@Override
public void finish() throws IOException {
parquetWriter.close();
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,60 @@
/*
* 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
*
* https://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.flink.formats.parquet;

import org.apache.flink.annotation.PublicEvolving;
import org.apache.flink.api.common.serialization.BulkWriter;
import org.apache.flink.core.fs.FSDataOutputStream;

import org.apache.parquet.hadoop.ParquetWriter;
import org.apache.parquet.io.OutputFile;

import java.io.IOException;

/**
* A factory that creates a Parquet {@link BulkWriter}. The factory takes a user-supplied
* builder to assemble Parquet's writer and then turns it into a Flink {@code BulkWriter}.
*
* @param <T> The type of record to write.
*/
@PublicEvolving
public class ParquetWriterFactory<T> implements BulkWriter.Factory<T> {

private static final long serialVersionUID = 1L;

/** The builder to construct the ParquetWriter. */
private final ParquetBuilder<T> writerBuilder;

/**
* Creates a new ParquetWriterFactory using the given builder to assemble the
* ParquetWriter.
*
* @param writerBuilder The builder to construct the ParquetWriter.
*/
public ParquetWriterFactory(ParquetBuilder<T> writerBuilder) {
this.writerBuilder = writerBuilder;
}

@Override
public BulkWriter<T> create(FSDataOutputStream stream) throws IOException {
final OutputFile out = new StreamOutputFile(stream);
final ParquetWriter<T> writer = writerBuilder.createWriter(out);
return new ParquetBulkWriter<>(writer);
}
}
Loading

0 comments on commit 66b1f85

Please sign in to comment.