Skip to content

Commit

Permalink
[FLINK-10457][fs-connector] Add SequenceFile support to StreamingFile…
Browse files Browse the repository at this point in the history
…Sink.

This closes apache#6774.
  • Loading branch information
kl0u committed Dec 18, 2018
1 parent b078c80 commit cf88590
Show file tree
Hide file tree
Showing 10 changed files with 230 additions and 96 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -19,14 +19,15 @@
package org.apache.flink.formats.parquet.avro;

import org.apache.flink.api.common.typeinfo.TypeInformation;
import org.apache.flink.api.java.tuple.Tuple2;
import org.apache.flink.core.fs.Path;
import org.apache.flink.formats.avro.typeutils.GenericRecordAvroTypeInfo;
import org.apache.flink.formats.parquet.generated.Address;
import org.apache.flink.streaming.api.datastream.DataStream;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.streaming.api.functions.sink.filesystem.StreamingFileSink;
import org.apache.flink.streaming.util.FiniteTestSource;
import org.apache.flink.test.util.AbstractTestBase;
import org.apache.flink.test.util.FiniteTestSource;

import org.apache.avro.Schema;
import org.apache.avro.generic.GenericData;
Expand Down Expand Up @@ -156,11 +157,14 @@ private static <T> void validateResults(File folder, GenericData dataModel, List

File[] partFiles = buckets[0].listFiles();
assertNotNull(partFiles);
assertEquals(1, partFiles.length);
assertTrue(partFiles[0].length() > 0);
assertEquals(2, partFiles.length);

List<Address> results = readParquetFile(partFiles[0], dataModel);
assertEquals(expected, results);
for (File partFile : partFiles) {
assertTrue(partFile.length() > 0);

final List<Tuple2<Long, String>> fileContent = readParquetFile(partFile, dataModel);
assertEquals(expected, fileContent);
}
}

private static <T> List<T> readParquetFile(File file, GenericData dataModel) throws IOException {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -29,8 +29,8 @@ under the License.
<relativePath>..</relativePath>
</parent>

<artifactId>flink-sequencefile</artifactId>
<name>flink-sequencefile</name>
<artifactId>flink-sequence-file</artifactId>
<name>flink-sequence-file</name>

<packaging>jar</packaging>

Expand All @@ -50,13 +50,6 @@ under the License.
<version>${project.version}</version>
<scope>provided</scope>
</dependency>
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-hadoop-fs</artifactId>
<version>${project.version}</version>
<scope>provided</scope>
</dependency>


<!-- test dependencies -->

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -37,9 +37,10 @@
*/
@PublicEvolving
public class SequenceFileWriter<K extends Writable, V extends Writable> implements BulkWriter<Tuple2<K, V>> {

private final SequenceFile.Writer writer;

public SequenceFileWriter(SequenceFile.Writer writer) {
SequenceFileWriter(SequenceFile.Writer writer) {
this.writer = checkNotNull(writer);
}

Expand All @@ -58,4 +59,3 @@ public void finish() throws IOException {
writer.close();
}
}

Original file line number Diff line number Diff line change
Expand Up @@ -22,9 +22,6 @@
import org.apache.flink.api.common.serialization.BulkWriter;
import org.apache.flink.api.java.tuple.Tuple2;
import org.apache.flink.core.fs.FSDataOutputStream;
import org.apache.flink.core.fs.FileSystem;
import org.apache.flink.core.fs.Path;
import org.apache.flink.runtime.fs.hdfs.HadoopFileSystem;

import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.io.SequenceFile;
Expand All @@ -33,9 +30,6 @@
import org.apache.hadoop.io.compress.CompressionCodecFactory;

import java.io.IOException;
import java.io.ObjectInputStream;
import java.io.ObjectOutputStream;
import java.io.Serializable;

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

Expand All @@ -47,9 +41,13 @@
*/
@PublicEvolving
public class SequenceFileWriterFactory<K extends Writable, V extends Writable> implements BulkWriter.Factory<Tuple2<K, V>> {

private static final long serialVersionUID = 1L;

private final SerializableHadoopConfiguration serdeHadoopConfig;
/** A constant specifying that no compression is requested. */
public static final String NO_COMPRESSION = "NO_COMPRESSION";

private final SerializableHadoopConfiguration serializableHadoopConfig;
private final Class<K> keyClass;
private final Class<V> valueClass;
private final String compressionCodecName;
Expand All @@ -64,7 +62,7 @@ public class SequenceFileWriterFactory<K extends Writable, V extends Writable> i
* @param valueClass The class of value to write.
*/
public SequenceFileWriterFactory(Configuration hadoopConf, Class<K> keyClass, Class<V> valueClass) {
this(hadoopConf, keyClass, valueClass, "None", SequenceFile.CompressionType.BLOCK);
this(hadoopConf, keyClass, valueClass, NO_COMPRESSION, SequenceFile.CompressionType.BLOCK);
}

/**
Expand All @@ -91,7 +89,7 @@ public SequenceFileWriterFactory(Configuration hadoopConf, Class<K> keyClass, Cl
* @param compressionType The type of compression level.
*/
public SequenceFileWriterFactory(Configuration hadoopConf, Class<K> keyClass, Class<V> valueClass, String compressionCodecName, SequenceFile.CompressionType compressionType) {
this.serdeHadoopConfig = new SerializableHadoopConfiguration(checkNotNull(hadoopConf));
this.serializableHadoopConfig = new SerializableHadoopConfiguration(checkNotNull(hadoopConf));
this.keyClass = checkNotNull(keyClass);
this.valueClass = checkNotNull(valueClass);
this.compressionCodecName = checkNotNull(compressionCodecName);
Expand All @@ -101,9 +99,9 @@ public SequenceFileWriterFactory(Configuration hadoopConf, Class<K> keyClass, Cl
@Override
public SequenceFileWriter<K, V> create(FSDataOutputStream out) throws IOException {
org.apache.hadoop.fs.FSDataOutputStream stream = new org.apache.hadoop.fs.FSDataOutputStream(out, null);
CompressionCodec compressionCodec = getCompressionCodec(serdeHadoopConfig.get(), compressionCodecName);
CompressionCodec compressionCodec = getCompressionCodec(serializableHadoopConfig.get(), compressionCodecName);
SequenceFile.Writer writer = SequenceFile.createWriter(
serdeHadoopConfig.get(),
serializableHadoopConfig.get(),
SequenceFile.Writer.stream(stream),
SequenceFile.Writer.keyClass(keyClass),
SequenceFile.Writer.valueClass(valueClass),
Expand All @@ -112,57 +110,19 @@ public SequenceFileWriter<K, V> create(FSDataOutputStream out) throws IOExceptio
}

private CompressionCodec getCompressionCodec(Configuration conf, String compressionCodecName) {
if (compressionCodecName.equals("None")) {
checkNotNull(conf);
checkNotNull(compressionCodecName);

if (compressionCodecName.equals(NO_COMPRESSION)) {
return null;
}

CompressionCodecFactory codecFactory = new CompressionCodecFactory(checkNotNull(conf));
CompressionCodecFactory codecFactory = new CompressionCodecFactory(conf);
CompressionCodec codec = codecFactory.getCodecByName(compressionCodecName);
if (codec == null) {
throw new RuntimeException("Codec " + compressionCodecName + " not found.");
}
return codec;
}

/**
* Get Hadoop configuration based by the path.
* If the path is not Hadoop URI, it will be return default configuration.
*
* @param path The path to get configuration.
* @return Hadoop configuration.
* @throws IOException
*/
public static Configuration getHadoopConfigFromPath(Path path) throws IOException {
FileSystem fs = FileSystem.getUnguardedFileSystem(path.toUri());
if (fs != null && fs instanceof HadoopFileSystem) {
return ((HadoopFileSystem) fs).getHadoopFileSystem().getConf();
} else {
return new Configuration();
}
}

/**
* The wrapper class for serialization of {@link Configuration}.
*/
private class SerializableHadoopConfiguration implements Serializable {
private transient Configuration hadoopConfig;

private SerializableHadoopConfiguration(Configuration hadoopConfig) {
this.hadoopConfig = hadoopConfig;
}

private Configuration get() {
return this.hadoopConfig;
}

private void writeObject(ObjectOutputStream out) throws IOException {
this.hadoopConfig.write(out);
}

private void readObject(ObjectInputStream in) throws IOException {
this.hadoopConfig = new Configuration();
this.hadoopConfig.readFields(in);
}
}
}

Original file line number Diff line number Diff line change
@@ -0,0 +1,58 @@
/*
* 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: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.sequencefile;

import org.apache.hadoop.conf.Configuration;

import java.io.IOException;
import java.io.ObjectInputStream;
import java.io.ObjectOutputStream;
import java.io.Serializable;

/**
* Wrapper class for serialization of {@link Configuration}.
*/
class SerializableHadoopConfiguration implements Serializable {

private static final long serialVersionUID = 1L;

private transient Configuration hadoopConfig;

SerializableHadoopConfiguration(Configuration hadoopConfig) {
this.hadoopConfig = hadoopConfig;
}

Configuration get() {
return this.hadoopConfig;
}

// --------------------
private void writeObject(ObjectOutputStream out) throws IOException {
this.hadoopConfig.write(out);
}

private void readObject(ObjectInputStream in) throws IOException {
final Configuration config = new Configuration();
config.readFields(in);

if (this.hadoopConfig == null) {
this.hadoopConfig = config;
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -26,8 +26,8 @@
import org.apache.flink.streaming.api.datastream.DataStream;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.streaming.api.functions.sink.filesystem.StreamingFileSink;
import org.apache.flink.streaming.util.FiniteTestSource;
import org.apache.flink.test.util.AbstractTestBase;
import org.apache.flink.test.util.FiniteTestSource;

import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.io.LongWritable;
Expand All @@ -49,26 +49,31 @@
* Integration test case for writing bulk encoded files with the
* {@link StreamingFileSink} with SequenceFile.
*/
public class SequenceFileSinkITCase extends AbstractTestBase {
@Test
public void testWriteSequenceFile() throws Exception {
final File folder = TEMPORARY_FOLDER.newFolder();
public class SequenceStreamingFileSinkITCase extends AbstractTestBase {

private final Configuration configuration = new Configuration();

final List<Tuple2<Long, String>> data = Arrays.asList(
private final List<Tuple2<Long, String>> testData = Arrays.asList(
new Tuple2<>(1L, "a"),
new Tuple2<>(2L, "b"),
new Tuple2<>(3L, "c")
);
);

@Test
public void testWriteSequenceFile() throws Exception {
final File folder = TEMPORARY_FOLDER.newFolder();
final Path testPath = Path.fromLocalFile(folder);

final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
env.setParallelism(1);
env.enableCheckpointing(100);

DataStream<Tuple2<Long, String>> stream = env.addSource(
new FiniteTestSource<>(data), TypeInformation.of(new TypeHint<Tuple2<Long, String>>() {
}));
new FiniteTestSource<>(testData),
TypeInformation.of(new TypeHint<Tuple2<Long, String>>() {

Path testPath = Path.fromLocalFile(folder);
})
);

stream.map(new MapFunction<Tuple2<Long, String>, Tuple2<LongWritable, Text>>() {
@Override
Expand All @@ -78,17 +83,17 @@ public Tuple2<LongWritable, Text> map(Tuple2<Long, String> value) throws Excepti
}).addSink(
StreamingFileSink.forBulkFormat(
testPath,
new SequenceFileWriterFactory<>(SequenceFileWriterFactory.getHadoopConfigFromPath(testPath), LongWritable.class, Text.class, "BZip2")
new SequenceFileWriterFactory<>(configuration, LongWritable.class, Text.class, "BZip2")
).build());

env.execute();

validateResults(folder, data);
validateResults(folder, testData);
}

private List<Tuple2<Long, String>> readSequenceFile(File file) throws IOException {
SequenceFile.Reader reader = new SequenceFile.Reader(
new Configuration(), SequenceFile.Reader.file(new org.apache.hadoop.fs.Path(file.toURI())));
configuration, SequenceFile.Reader.file(new org.apache.hadoop.fs.Path(file.toURI())));
LongWritable key = new LongWritable();
Text val = new Text();
ArrayList<Tuple2<Long, String>> results = new ArrayList<>();
Expand All @@ -104,14 +109,15 @@ private void validateResults(File folder, List<Tuple2<Long, String>> expected) t
assertNotNull(buckets);
assertEquals(1, buckets.length);

File[] partFiles = buckets[0].listFiles();
final File[] partFiles = buckets[0].listFiles();
assertNotNull(partFiles);
assertEquals(1, partFiles.length);
assertTrue(partFiles[0].length() > 0);
assertEquals(2, partFiles.length);

for (File partFile : partFiles) {
assertTrue(partFile.length() > 0);

List<Tuple2<Long, String>> results = readSequenceFile(partFiles[0]);
assertEquals(expected, results);
final List<Tuple2<Long, String>> fileContent = readSequenceFile(partFile);
assertEquals(expected, fileContent);
}
}
}


Loading

0 comments on commit cf88590

Please sign in to comment.