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

Add Java Parquet read/write pipelines #50

Merged
merged 4 commits into from
Feb 21, 2024
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
24 changes: 24 additions & 0 deletions Java/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -347,6 +347,13 @@
<artifactId>beam-sdks-java-extensions-avro</artifactId>
</dependency>

<!-- Adds a dependency on the Beam Parquet IO module. -->
<dependency>
<groupId>org.apache.beam</groupId>
<artifactId>beam-sdks-java-io-parquet</artifactId>
<version>${beam.version}</version>
</dependency>

<!-- Dependencies below this line are specific dependencies needed by the examples code. -->
<dependency>
<groupId>com.google.api-client</groupId>
Expand Down Expand Up @@ -408,6 +415,23 @@
<version>${joda.version}</version>
</dependency>

<!-- Adds a dependency on the Hadoop Client module. -->
<dependency>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-client</artifactId>
<version>${hadoop.version}</version>
<exclusions>
<exclusion>
<groupId>org.slf4j</groupId>
<artifactId>slf4j-log4j12</artifactId>
</exclusion>
<exclusion>
<groupId>org.slf4j</groupId>
<artifactId>slf4j-jdk14</artifactId>
</exclusion>
</exclusions>
</dependency>

<!-- Adds a dependency for the JDBC. -->
<dependency>
<groupId>org.postgresql</groupId>
Expand Down
14 changes: 6 additions & 8 deletions Java/src/main/java/jdbc/ReadPartitionsJdbc.java
Original file line number Diff line number Diff line change
@@ -1,13 +1,11 @@
/*
* 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
* Copyright 2024 Google LLC
*
* http://www.apache.org/licenses/LICENSE-2.0
* 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
*
* 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,
Expand Down
14 changes: 6 additions & 8 deletions Java/src/main/java/jdbc/WriteJdbc.java
Original file line number Diff line number Diff line change
@@ -1,13 +1,11 @@
/*
* 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
* Copyright 2024 Google LLC
*
* http://www.apache.org/licenses/LICENSE-2.0
* 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
*
* 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,
Expand Down
86 changes: 86 additions & 0 deletions Java/src/main/java/parquet/ReadParquet.java
Original file line number Diff line number Diff line change
@@ -0,0 +1,86 @@
/*
* Copyright 2024 Google LLC
*
* 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
*
* 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 parquet;

import java.io.Serializable;
import java.util.Objects;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericData;
import org.apache.avro.generic.GenericRecord;
import org.apache.avro.reflect.ReflectData;
import org.apache.beam.sdk.Pipeline;
import org.apache.beam.sdk.io.parquet.ParquetIO;
import org.apache.beam.sdk.options.Description;
import org.apache.beam.sdk.options.PipelineOptions;
import org.apache.beam.sdk.options.PipelineOptionsFactory;
import org.apache.beam.sdk.options.Validation;
import org.apache.beam.sdk.transforms.DoFn;
import org.apache.beam.sdk.transforms.ParDo;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

public class ReadParquet {

private static final Logger LOG = LoggerFactory.getLogger(ReadParquet.class);

/** Represents an Example Parquet record. */
public static class ExampleRecord implements Serializable {
public static final String ID_COLUMN = "id";
public static final String NAME_COLUMN = "name";

private int id;
private String name;
}

/** Pipeline options for read from Parquet files. */
public interface ReadParquetOptions extends PipelineOptions {
@Description("A file glob pattern to read Parquet from")
@Validation.Required
String getFilePattern();

void setFilePattern(String filePattern);
}

public static void main(String[] args) {
ReadParquetOptions options =
PipelineOptionsFactory.fromArgs(args).withValidation().as(ReadParquetOptions.class);

Schema exampleRecordSchema = ReflectData.get().getSchema(ExampleRecord.class);

Pipeline p = Pipeline.create(options);

p.apply(
"Read from Parquet",
ParquetIO.read(exampleRecordSchema)
.withAvroDataModel(GenericData.get())
.from(options.getFilePattern()))
.apply(
"Log Data",
ParDo.of(
new DoFn<GenericRecord, GenericRecord>() {
@ProcessElement
public void processElement(ProcessContext c) {
GenericRecord record = Objects.requireNonNull(c.element());
LOG.info(
"Id = {}, Name = {}",
record.get(ExampleRecord.ID_COLUMN),
record.get(ExampleRecord.NAME_COLUMN));
c.output(record);
}
}));
p.run();
}
}
125 changes: 125 additions & 0 deletions Java/src/main/java/parquet/WriteParquet.java
Original file line number Diff line number Diff line change
@@ -0,0 +1,125 @@
/*
* Copyright 2024 Google LLC
*
* 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
*
* 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 parquet;

import java.io.Serializable;
import java.util.Arrays;
import java.util.List;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericRecord;
import org.apache.avro.generic.GenericRecordBuilder;
import org.apache.avro.reflect.ReflectData;
import org.apache.beam.sdk.Pipeline;
import org.apache.beam.sdk.extensions.avro.coders.AvroCoder;
import org.apache.beam.sdk.io.FileIO;
import org.apache.beam.sdk.io.parquet.ParquetIO;
import org.apache.beam.sdk.options.Description;
import org.apache.beam.sdk.options.PipelineOptions;
import org.apache.beam.sdk.options.PipelineOptionsFactory;
import org.apache.beam.sdk.options.Validation;
import org.apache.beam.sdk.transforms.Create;
import org.apache.beam.sdk.transforms.MapElements;
import org.apache.beam.sdk.transforms.SimpleFunction;

public class WriteParquet {

/** Represents an Example Parquet record. */
public static class ExampleRecord implements Serializable {
public static final String ID_COLUMN = "id";
public static final String NAME_COLUMN = "name";

private int id;
private String name;

public ExampleRecord() {}

public ExampleRecord(int id, String name) {
this.id = id;
this.name = name;
}

public int getId() {
return id;
}

public String getName() {
return name;
}
}

/** Pipeline options for write to Parquet files. */
public interface WriteParquetOptions extends PipelineOptions {
@Description("A file path to write Parquet files to")
@Validation.Required
String getFilePath();

void setFilePath(String filePath);
}

public static void main(String[] args) {
WriteParquetOptions options =
PipelineOptionsFactory.fromArgs(args).withValidation().as(WriteParquetOptions.class);

Schema exampleRecordSchema = ReflectData.get().getSchema(ExampleRecord.class);

Pipeline p = Pipeline.create(options);

List<ExampleRecord> rows =
Arrays.asList(
new ExampleRecord(1, "Charles"),
new ExampleRecord(2, "Alice"),
new ExampleRecord(3, "Bob"),
new ExampleRecord(4, "Amanda"),
new ExampleRecord(5, "Alex"),
new ExampleRecord(6, "Eliza"));

p.apply("Create", Create.of(rows))
.apply(
"Map to GenericRecord",
MapElements.via(new MapExampleRecordToGenericRecord(exampleRecordSchema)))
.setCoder(AvroCoder.of(exampleRecordSchema))
.apply(
"Write to Parquet",
FileIO.<GenericRecord>write()
.via(ParquetIO.sink(exampleRecordSchema))
.to(options.getFilePath()));
p.run();
}

/**
* Converts instances of {@link ExampleRecord} to {@link GenericRecord} using a provided Avro
* schema.
*/
private static class MapExampleRecordToGenericRecord
extends SimpleFunction<ExampleRecord, GenericRecord> {

private final Schema schema;

public MapExampleRecordToGenericRecord(Schema schema) {
this.schema = schema;
}

@Override
public GenericRecord apply(ExampleRecord input) {
GenericRecordBuilder builder = new GenericRecordBuilder(schema);

builder
.set(ExampleRecord.NAME_COLUMN, input.getName())
.set(ExampleRecord.ID_COLUMN, input.getId());
return builder.build();
}
}
}
Loading