Skip to content

Commit

Permalink
Add ability to handle streaming input to AvroSchemaIOProvider (apache…
Browse files Browse the repository at this point in the history
  • Loading branch information
laraschmidt committed Mar 23, 2022
1 parent b61436e commit b9846fb
Show file tree
Hide file tree
Showing 2 changed files with 207 additions and 10 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -21,17 +21,24 @@
import java.io.Serializable;
import org.apache.avro.generic.GenericRecord;
import org.apache.beam.sdk.annotations.Internal;
import org.apache.beam.sdk.io.AvroIO.Write;
import org.apache.beam.sdk.schemas.Schema;
import org.apache.beam.sdk.schemas.Schema.FieldType;
import org.apache.beam.sdk.schemas.io.SchemaIO;
import org.apache.beam.sdk.schemas.io.SchemaIOProvider;
import org.apache.beam.sdk.schemas.transforms.Convert;
import org.apache.beam.sdk.schemas.utils.AvroUtils;
import org.apache.beam.sdk.transforms.PTransform;
import org.apache.beam.sdk.transforms.windowing.FixedWindows;
import org.apache.beam.sdk.transforms.windowing.Window;
import org.apache.beam.sdk.values.PBegin;
import org.apache.beam.sdk.values.PCollection;
import org.apache.beam.sdk.values.PCollection.IsBounded;
import org.apache.beam.sdk.values.PDone;
import org.apache.beam.sdk.values.POutput;
import org.apache.beam.sdk.values.Row;
import org.checkerframework.checker.nullness.qual.Nullable;
import org.joda.time.Duration;

/**
* An implementation of {@link SchemaIOProvider} for reading and writing Avro files with {@link
Expand All @@ -55,7 +62,7 @@ public String identifier() {
*/
@Override
public Schema configurationSchema() {
return Schema.builder().build();
return Schema.builder().addNullableField("writeWindowSizeSeconds", FieldType.INT64).build();
}

/**
Expand All @@ -64,7 +71,7 @@ public Schema configurationSchema() {
*/
@Override
public AvroSchemaIO from(String location, Row configuration, Schema dataSchema) {
return new AvroSchemaIO(location, dataSchema);
return new AvroSchemaIO(location, dataSchema, configuration);
}

@Override
Expand All @@ -74,17 +81,25 @@ public boolean requiresDataSchema() {

@Override
public PCollection.IsBounded isBounded() {
// This supports streaming now as well but there's no option for this. The move to
// SchemaTransform will remove the need to provide this.
return PCollection.IsBounded.BOUNDED;
}

/** An abstraction to create schema aware IOs. */
private static class AvroSchemaIO implements SchemaIO, Serializable {
protected final Schema dataSchema;
protected final String location;
protected final @Nullable Duration windowSize;

private AvroSchemaIO(String location, Schema dataSchema) {
private AvroSchemaIO(String location, Schema dataSchema, Row configuration) {
this.dataSchema = dataSchema;
this.location = location;
if (configuration.getInt64("writeWindowSizeSeconds") != null) {
windowSize = Duration.standardSeconds(configuration.getInt64("writeWindowSizeSeconds"));
} else {
windowSize = null;
}
}

@Override
Expand Down Expand Up @@ -113,13 +128,22 @@ public PTransform<PCollection<Row>, POutput> buildWriter() {
return new PTransform<PCollection<Row>, POutput>() {
@Override
public PDone expand(PCollection<Row> input) {
return input
.apply("ToGenericRecords", Convert.to(GenericRecord.class))
.apply(
"AvroIOWrite",
AvroIO.writeGenericRecords(AvroUtils.toAvroSchema(dataSchema, null, null))
.to(location)
.withoutSharding());
PCollection<GenericRecord> asRecords =
input.apply("ToGenericRecords", Convert.to(GenericRecord.class));
Write<GenericRecord> avroWrite =
AvroIO.writeGenericRecords(AvroUtils.toAvroSchema(dataSchema, null, null))
.to(location);
if (input.isBounded() == IsBounded.UNBOUNDED || windowSize != null) {
asRecords =
asRecords.apply(
Window.into(
FixedWindows.of(
windowSize == null ? Duration.standardMinutes(1) : windowSize)));
avroWrite = avroWrite.withWindowedWrites().withNumShards(1);
} else {
avroWrite = avroWrite.withoutSharding();
}
return asRecords.apply("AvroIOWrite", avroWrite);
}
};
}
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,173 @@
/*
* 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;

import static org.junit.Assert.assertEquals;

import java.io.File;
import java.time.Duration;
import java.util.Arrays;
import java.util.List;
import org.apache.beam.sdk.coders.RowCoder;
import org.apache.beam.sdk.io.fs.MatchResult;
import org.apache.beam.sdk.schemas.Schema;
import org.apache.beam.sdk.schemas.io.SchemaIO;
import org.apache.beam.sdk.testing.NeedsRunner;
import org.apache.beam.sdk.testing.PAssert;
import org.apache.beam.sdk.testing.TestPipeline;
import org.apache.beam.sdk.testing.TestStream;
import org.apache.beam.sdk.transforms.Create;
import org.apache.beam.sdk.values.PCollection;
import org.apache.beam.sdk.values.Row;
import org.apache.beam.sdk.values.TimestampedValue;
import org.joda.time.Instant;
import org.junit.Rule;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import org.junit.rules.TemporaryFolder;
import org.junit.runner.RunWith;
import org.junit.runners.JUnit4;

/** Test for AvroSchemaIOProvider. */
@RunWith(JUnit4.class)
public class AvroSchemaIOProviderTest {
@Rule public TestPipeline writePipeline = TestPipeline.create();
@Rule public TestPipeline readPipeline = TestPipeline.create();
@Rule public TemporaryFolder tempFolder = new TemporaryFolder();

private static final Schema SCHEMA =
Schema.builder().addInt64Field("age").addStringField("age_str").build();

private Row createRow(long l) {
return Row.withSchema(SCHEMA).addValues(l, Long.valueOf(l).toString()).build();
}

@Test
@Category({NeedsRunner.class})
public void testWriteAndReadTable() {
File destinationFile = new File(tempFolder.getRoot(), "person-info.avro");

AvroSchemaIOProvider provider = new AvroSchemaIOProvider();
Row configuration = Row.withSchema(provider.configurationSchema()).addValue(null).build();
SchemaIO io = provider.from(destinationFile.getAbsolutePath(), configuration, SCHEMA);

List<Row> rowsList = Arrays.asList(createRow(1L), createRow(3L), createRow(4L));
PCollection<Row> rows =
writePipeline.apply("Create", Create.of(rowsList).withCoder(RowCoder.of(SCHEMA)));
rows.apply(io.buildWriter());
writePipeline.run();

PCollection<Row> read = readPipeline.begin().apply(io.buildReader());
PAssert.that(read).containsInAnyOrder(rowsList);
readPipeline.run();
}

@Test
@Category({NeedsRunner.class})
public void testStreamingWriteDefault() throws Exception {
File destinationFile = new File(tempFolder.getRoot(), "person-info");

AvroSchemaIOProvider provider = new AvroSchemaIOProvider();
Row config = Row.withSchema(provider.configurationSchema()).addValue(null).build();
SchemaIO writeIO = provider.from(destinationFile.getAbsolutePath(), config, SCHEMA);

TestStream<Row> createEvents =
TestStream.create(RowCoder.of(SCHEMA))
.addElements(TimestampedValue.of(createRow(1L), new Instant(1L)))
.addElements(TimestampedValue.of(createRow(2L), Instant.ofEpochSecond(120L)))
.advanceWatermarkToInfinity();

writePipeline.apply("create", createEvents).apply("write", writeIO.buildWriter());
writePipeline.run();

// Verify we wrote two files.
String wildcardPath = destinationFile.getAbsolutePath() + "*";
MatchResult result = FileSystems.match(wildcardPath);
assertEquals(2, result.metadata().size());

// Verify results of the files.
SchemaIO readIO = provider.from(wildcardPath, config, SCHEMA);
PCollection<Row> read = readPipeline.begin().apply("read", readIO.buildReader());
PAssert.that(read).containsInAnyOrder(createRow(1L), createRow(2L));
readPipeline.run();
}

@Test
@Category({NeedsRunner.class})
public void testStreamingCustomWindowSize() throws Exception {
File destinationFile = new File(tempFolder.getRoot(), "person-info");

AvroSchemaIOProvider provider = new AvroSchemaIOProvider();
Row config =
Row.withSchema(provider.configurationSchema())
.addValue(Duration.ofMinutes(4).getSeconds())
.build();
SchemaIO writeIO = provider.from(destinationFile.getAbsolutePath(), config, SCHEMA);

TestStream<Row> createEvents =
TestStream.create(RowCoder.of(SCHEMA))
.addElements(TimestampedValue.of(createRow(1L), new Instant(1L)))
.addElements(TimestampedValue.of(createRow(2L), Instant.ofEpochSecond(120L)))
.advanceWatermarkToInfinity();

writePipeline.apply("create", createEvents).apply("write", writeIO.buildWriter());
writePipeline.run();

// Verify we wrote one file.
String wildcardPath = destinationFile.getAbsolutePath() + "*";
MatchResult result = FileSystems.match(wildcardPath);
assertEquals(1, result.metadata().size());

// Verify results of the files.
SchemaIO readIO = provider.from(wildcardPath, config, SCHEMA);
PCollection<Row> read = readPipeline.begin().apply("read", readIO.buildReader());
PAssert.that(read).containsInAnyOrder(createRow(1L), createRow(2L));
readPipeline.run();
}

@Test
@Category({NeedsRunner.class})
public void testBatchCustomWindowSize() throws Exception {
File destinationFile = new File(tempFolder.getRoot(), "person-info");

AvroSchemaIOProvider provider = new AvroSchemaIOProvider();
Row config =
Row.withSchema(provider.configurationSchema())
.addValue(Duration.ofMinutes(4).getSeconds())
.build();
SchemaIO writeIO = provider.from(destinationFile.getAbsolutePath(), config, SCHEMA);

List<Row> rowsList = Arrays.asList(createRow(1L), createRow(3L), createRow(4L));
PCollection<Row> rows =
writePipeline.apply("Create", Create.of(rowsList).withCoder(RowCoder.of(SCHEMA)));

rows.apply("write", writeIO.buildWriter());
writePipeline.run();

// Verify we wrote one file.
String wildcardPath = destinationFile.getAbsolutePath() + "*";
MatchResult result = FileSystems.match(wildcardPath);
assertEquals(1, result.metadata().size());

// Verify results of the files.
SchemaIO readIO = provider.from(wildcardPath, config, SCHEMA);
PCollection<Row> read = readPipeline.begin().apply("read", readIO.buildReader());
PAssert.that(read).containsInAnyOrder(rowsList);
readPipeline.run();
}
}

0 comments on commit b9846fb

Please sign in to comment.