Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[#1459] Added data sink/source for Parquet #1586

Merged
merged 10 commits into from
Nov 22, 2023
Original file line number Diff line number Diff line change
Expand Up @@ -52,4 +52,8 @@
files="[/\\]src[/\\]test[/\\]java[/\\]"/>
<suppress checks="StaticVariableName"
files="[/\\]src[/\\]test[/\\]java[/\\]"/>

<!-- no checkstyle for auto generated code -->
<suppress checks=".*"
files="[/\\]target[/\\]generated-sources[/\\]"/>
</suppressions>
10 changes: 10 additions & 0 deletions gradoop-flink/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -54,6 +54,10 @@

<build>
<plugins>
<plugin>
<groupId>com.github.os72</groupId>
<artifactId>protoc-jar-maven-plugin</artifactId>
</plugin>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-compiler-plugin</artifactId>
Expand Down Expand Up @@ -124,6 +128,12 @@
<artifactId>flink-shaded-hadoop-2</artifactId>
</dependency>

<!-- Parquet -->
<dependency>
<groupId>org.apache.parquet</groupId>
<artifactId>parquet-protobuf</artifactId>
</dependency>

<!-- Test dependencies -->

<!-- Gradoop -->
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,77 @@
/*
* Copyright © 2014 - 2021 Leipzig University (Database Research Group)
*
* Licensed 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.gradoop.flink.io.impl.parquet.common;

import org.apache.flink.api.common.typeinfo.TypeInformation;
import org.apache.flink.api.java.hadoop.mapreduce.HadoopInputFormatBase;
import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
import org.apache.hadoop.mapreduce.InputFormat;
import org.apache.hadoop.mapreduce.Job;

import java.io.IOException;

/**
* InputFormat implementation allowing to use Hadoop (mapreduce) InputFormats which don't provide a key with
* Flink.
*
* @param <T> Value Type
*/
public class HadoopValueInputFormat<T> extends HadoopInputFormatBase<Void, T, T> implements
ResultTypeQueryable<T> {

/**
* Creates a new Flink input format.
*
* @param mapreduceInputFormat Hadoop (mapreduce) input format
* @param value value type class
* @param job job instance for configuration
*/
public HadoopValueInputFormat(InputFormat<Void, T> mapreduceInputFormat, Class<T> value, Job job) {
super(mapreduceInputFormat, Void.class, value, job);
}

@Override
public T nextRecord(T record) throws IOException {
if (!this.fetched) {
fetchNext();
}
if (!this.hasNext) {
return null;
}
try {
record = recordReader.getCurrentValue();
} catch (InterruptedException e) {
throw new IOException("Could not get KeyValue pair.", e);
}
this.fetched = false;

return record;
}

@Override
public TypeInformation<T> getProducedType() {
return TypeInformation.of(this.valueClass);
}

@Override
public String toString() {
String jobName = this.getConfiguration().get("mapreduce.job.name");
if (jobName != null) {
return String.format("HadoopValueInputFormat[%s]", jobName);
}
return super.toString();
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,59 @@
/*
* Copyright © 2014 - 2021 Leipzig University (Database Research Group)
*
* Licensed 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.gradoop.flink.io.impl.parquet.common;

import org.apache.flink.api.java.hadoop.mapreduce.HadoopOutputFormatBase;
import org.apache.hadoop.mapreduce.Job;
import org.apache.hadoop.mapreduce.OutputFormat;

import java.io.IOException;

/**
* OutputFormat implementation allowing to use Hadoop (mapreduce) OutputFormats which don't provide a key
* with Flink.
*
* @param <T> Value Type
*/
public class HadoopValueOutputFormat<T> extends HadoopOutputFormatBase<Void, T, T> {

/**
* Creates a new Flink output format.
*
* @param mapreduceOutputFormat Hadoop (mapreduce) output format
* @param job job instance for configuration
*/
public HadoopValueOutputFormat(OutputFormat<Void, T> mapreduceOutputFormat, Job job) {
super(mapreduceOutputFormat, job);
}

@Override
public void writeRecord(T record) throws IOException {
try {
this.recordWriter.write(null, record);
} catch (InterruptedException e) {
throw new IOException("Could not write Record.", e);
}
}

@Override
public String toString() {
String jobName = this.getConfiguration().get("mapreduce.job.name");
if (jobName != null) {
return String.format("HadoopValueOutputFormat[%s]", jobName);
}
return super.toString();
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,64 @@
/*
* Copyright © 2014 - 2021 Leipzig University (Database Research Group)
*
* Licensed 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.gradoop.flink.io.impl.parquet.common;

import org.apache.hadoop.mapreduce.JobContext;
import org.apache.hadoop.mapreduce.RecordWriter;
import org.apache.hadoop.mapreduce.TaskAttemptContext;
import org.apache.parquet.hadoop.ParquetFileWriter;
import org.apache.parquet.hadoop.ParquetOutputFormat;
import org.apache.parquet.hadoop.util.ContextUtil;

import java.io.IOException;

/**
* Extension of the {@link ParquetOutputFormat<T>} that allows specifying a file creation mode.
*
* @param <T> the type of the materialized records
*/
public class ParquetOutputFormatWithMode<T> extends ParquetOutputFormat<T> {

/**
* File creation mode configuration key
*/
private static final String PARQUET_WRITER_MODE = "parquet.writer.mode";

/**
* Sets the file creation mode of a {@link JobContext}.
*
* @param context the job's context
* @param mode the file creation mode
*/
public static void setFileCreationMode(JobContext context, ParquetFileWriter.Mode mode) {
ContextUtil.getConfiguration(context).setEnum(PARQUET_WRITER_MODE, mode);
}

/**
* Gets the file creation mode of a {@link JobContext}.
*
* @param context the job's context
* @return the specified mode or defaults to {@link ParquetFileWriter.Mode#CREATE} if not found
*/
public static ParquetFileWriter.Mode getFileCreationMode(JobContext context) {
return ContextUtil.getConfiguration(context).getEnum(PARQUET_WRITER_MODE, ParquetFileWriter.Mode.CREATE);
}

@Override
public RecordWriter<Void, T> getRecordWriter(TaskAttemptContext taskAttemptContext) throws IOException,
InterruptedException {
return super.getRecordWriter(taskAttemptContext, getFileCreationMode(taskAttemptContext));
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,20 @@
/*
* Copyright © 2014 - 2021 Leipzig University (Database Research Group)
*
* Licensed 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.
*/

/**
* Contains all commonly used classes related to parquet input and output to Flink.
*/
package org.gradoop.flink.io.impl.parquet.common;
Original file line number Diff line number Diff line change
@@ -0,0 +1,19 @@
/*
* Copyright © 2014 - 2021 Leipzig University (Database Research Group)
*
* Licensed 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.
*/
/**
* Contains all classes related to parquet input and output to Flink.
*/
package org.gradoop.flink.io.impl.parquet;
Loading
Loading