Skip to content

Commit

Permalink
Set avro schema configuration in format bundle
Browse files Browse the repository at this point in the history
Rather than managing the avro reader schema configuration in the input
format getSplits method, it needs to be managed when creating the format
bundle. Otherwise a crunch pipeline that has multiple inputs (kite views)
with different schemas will not see the correct reader schemas.

Note that the test only demonstrates the problem when also upgrading to
crunch 0.13.0 (which is not part of this commit). This is due to
CRUNCH-551 which is a fix for a problem in crunch that hides the current
issue (at least in the scenario of the test) in versions before crunch-0.13.0.

A test was also added to verify the behaviour with plain map/reduce to
ensure that this continues to work as expected.
  • Loading branch information
janvanbesien committed Jul 26, 2018
1 parent 1070ff9 commit 46068dc
Show file tree
Hide file tree
Showing 5 changed files with 222 additions and 86 deletions.
Original file line number Diff line number Diff line change
@@ -0,0 +1,55 @@
/**
* Copyright 2014 Cloudera Inc.
*
* 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.kitesdk.data.spi.filesystem;

import org.apache.avro.Schema;
import org.apache.avro.generic.GenericData;
import org.apache.avro.hadoop.io.AvroSerialization;
import org.apache.hadoop.conf.Configuration;
import org.apache.parquet.avro.AvroReadSupport;
import org.kitesdk.compat.DynMethods;
import org.kitesdk.data.Format;
import org.kitesdk.data.Formats;
import org.kitesdk.data.spi.DataModelUtil;

public class AvroConfigurationUtil {

// Constant from AvroJob copied here so we can set it on the Configuration
// given to this class.
private static final String AVRO_SCHEMA_INPUT_KEY = "avro.schema.input.key";

// this is required for 1.7.4 because setDataModelClass is not available
private static final DynMethods.StaticMethod setModel =
new DynMethods.Builder("setDataModelClass")
.impl(AvroSerialization.class, Configuration.class, Class.class)
.defaultNoop()
.buildStatic();

public static void configure(Configuration conf, Format format, Schema schema, Class<?> type) {
GenericData model = DataModelUtil.getDataModelForType(type);
if (Formats.AVRO.equals(format)) {
setModel.invoke(conf, model.getClass());
conf.set(AVRO_SCHEMA_INPUT_KEY, schema.toString());

} else if (Formats.PARQUET.equals(format)) {
// TODO: update to a version of Parquet with setAvroDataSupplier
//AvroReadSupport.setAvroDataSupplier(conf,
// DataModelUtil.supplierClassFor(model));
AvroReadSupport.setAvroReadSchema(conf, schema);
}
}

}
Original file line number Diff line number Diff line change
Expand Up @@ -17,67 +17,45 @@

import com.google.common.collect.ImmutableList;
import com.google.common.collect.Lists;
import java.io.IOException;
import java.util.Iterator;
import java.util.List;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericData;
import org.apache.avro.hadoop.io.AvroSerialization;
import org.apache.avro.mapred.AvroKey;
import org.apache.avro.mapreduce.AvroJob;
import org.apache.avro.mapreduce.AvroKeyInputFormat;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.NullWritable;
import org.apache.hadoop.mapreduce.InputFormat;
import org.apache.hadoop.mapreduce.InputSplit;
import org.apache.hadoop.mapreduce.Job;
import org.apache.hadoop.mapreduce.JobContext;
import org.apache.hadoop.mapreduce.RecordReader;
import org.apache.hadoop.mapreduce.TaskAttemptContext;
import org.apache.hadoop.mapreduce.*;
import org.apache.hadoop.mapreduce.lib.input.CombineFileSplit;
import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
import org.kitesdk.compat.DynMethods;
import org.apache.parquet.avro.AvroParquetInputFormat;
import org.kitesdk.compat.Hadoop;
import org.kitesdk.data.Format;
import org.kitesdk.data.Formats;
import org.kitesdk.data.spi.AbstractKeyRecordReaderWrapper;
import org.kitesdk.data.spi.AbstractRefinableView;
import org.kitesdk.data.spi.DataModelUtil;
import org.kitesdk.data.spi.FilteredRecordReader;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.parquet.avro.AvroParquetInputFormat;
import org.apache.parquet.avro.AvroReadSupport;

import java.io.IOException;
import java.util.Iterator;
import java.util.List;

class FileSystemViewKeyInputFormat<E> extends InputFormat<E, Void> {

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

// Constant from AvroJob copied here so we can set it on the Configuration
// given to this class.
private static final String AVRO_SCHEMA_INPUT_KEY = "avro.schema.input.key";

// this is required for 1.7.4 because setDataModelClass is not available
private static final DynMethods.StaticMethod setModel =
new DynMethods.Builder("setDataModelClass")
.impl(AvroSerialization.class, Configuration.class, Class.class)
.defaultNoop()
.buildStatic();

private FileSystemDataset<E> dataset;
private FileSystemView<E> view;

public FileSystemViewKeyInputFormat(FileSystemDataset<E> dataset,
Configuration conf) {
Configuration conf) {
this.dataset = dataset;
this.view = null;
LOG.debug("Dataset: {}", dataset);

Format format = dataset.getDescriptor().getFormat();

setConfigProperties(conf, format, dataset.getSchema(), dataset.getType());
AvroConfigurationUtil.configure(conf, format, dataset.getSchema(), dataset.getType());
}

public FileSystemViewKeyInputFormat(FileSystemView<E> view, Configuration conf) {
Expand All @@ -87,22 +65,7 @@ public FileSystemViewKeyInputFormat(FileSystemView<E> view, Configuration conf)

Format format = dataset.getDescriptor().getFormat();

setConfigProperties(conf, format, view.getSchema(), view.getType());
}

private static void setConfigProperties(Configuration conf, Format format,
Schema schema, Class<?> type) {
GenericData model = DataModelUtil.getDataModelForType(type);
if (Formats.AVRO.equals(format)) {
setModel.invoke(conf, model.getClass());
conf.set(AVRO_SCHEMA_INPUT_KEY, schema.toString());

} else if (Formats.PARQUET.equals(format)) {
// TODO: update to a version of Parquet with setAvroDataSupplier
//AvroReadSupport.setAvroDataSupplier(conf,
// DataModelUtil.supplierClassFor(model));
AvroReadSupport.setAvroReadSchema(conf, schema);
}
AvroConfigurationUtil.configure(conf, format, view.getSchema(), view.getType());
}

@Override
Expand All @@ -114,7 +77,6 @@ public List<InputSplit> getSplits(JobContext jobContext) throws IOException {

if (setInputPaths(jobContext, job)) {
if (Formats.AVRO.equals(format)) {
AvroJob.setInputKeySchema(job, dataset.getDescriptor().getSchema());
AvroCombineInputFormat<E> delegate = new AvroCombineInputFormat<E>();
return delegate.getSplits(jobContext);
} else if (Formats.PARQUET.equals(format)) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -15,10 +15,6 @@
*/
package org.kitesdk.data.crunch;

import java.io.IOException;
import java.net.URI;
import java.util.Map;
import java.util.Set;
import com.google.common.collect.ImmutableSet;
import org.apache.avro.generic.GenericData;
import org.apache.crunch.ReadableData;
Expand All @@ -41,9 +37,15 @@
import org.kitesdk.data.mapreduce.DatasetKeyInputFormat;
import org.kitesdk.data.spi.LastModifiedAccessor;
import org.kitesdk.data.spi.SizeAccessor;
import org.kitesdk.data.spi.filesystem.AvroConfigurationUtil;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import java.io.IOException;
import java.net.URI;
import java.util.Map;
import java.util.Set;

class DatasetSourceTarget<E> extends DatasetTarget<E> implements ReadableSourceTarget<E> {

private static final Logger LOG = LoggerFactory
Expand Down Expand Up @@ -72,8 +74,10 @@ public DatasetSourceTarget(View<E> view, AvroType<E> avroType) {
this.view = view;
this.avroType = avroType;

Configuration temp = new Configuration(false /* use an empty conf */ );
Configuration temp = new Configuration(false /* use an empty conf */);
DatasetKeyInputFormat.configure(temp).readFrom(view);
AvroConfigurationUtil.configure(temp, view.getDataset().getDescriptor().getFormat(),
view.getSchema(), view.getDataset().getType());
this.formatBundle = inputBundle(temp);
}

Expand All @@ -85,7 +89,7 @@ public DatasetSourceTarget(URI uri, AvroType<E> avroType) {
private static <E> AvroType<E> toAvroType(View<E> view, Class<E> type) {
if (type.isAssignableFrom(GenericData.Record.class)) {
return (AvroType<E>) Avros.generics(
view.getDataset().getDescriptor().getSchema());
view.getDataset().getDescriptor().getSchema());
} else {
return Avros.records(type);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,54 +16,32 @@
package org.kitesdk.data.crunch;

import com.google.common.io.Files;
import java.io.IOException;
import java.net.URI;
import java.util.Arrays;
import java.util.Collection;

import org.apache.avro.Schema;
import org.apache.avro.SchemaBuilder;
import org.apache.avro.generic.GenericData;
import org.apache.avro.generic.GenericData.Record;
import org.apache.avro.generic.GenericRecord;
import org.apache.crunch.CrunchRuntimeException;
import org.apache.crunch.MapFn;
import org.apache.crunch.PCollection;
import org.apache.crunch.Pipeline;
import org.apache.crunch.Target;
import org.apache.crunch.*;
import org.apache.crunch.impl.mr.MRPipeline;
import org.apache.crunch.types.avro.Avros;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.junit.After;
import org.junit.Assert;
import org.junit.Assume;
import org.junit.Before;
import org.junit.Test;
import org.junit.*;
import org.junit.runner.RunWith;
import org.junit.runners.Parameterized;
import org.kitesdk.compat.Hadoop;
import org.kitesdk.data.Dataset;
import org.kitesdk.data.DatasetDescriptor;
import org.kitesdk.data.DatasetReader;
import org.kitesdk.data.DatasetWriter;
import org.kitesdk.data.Datasets;
import org.kitesdk.data.Formats;
import org.kitesdk.data.MiniDFSTest;
import org.kitesdk.data.Signalable;
import org.kitesdk.data.spi.PartitionKey;
import org.kitesdk.data.PartitionStrategy;
import org.kitesdk.data.*;
import org.kitesdk.data.spi.DatasetRepository;
import org.kitesdk.data.spi.PartitionedDataset;
import org.kitesdk.data.View;
import org.kitesdk.data.spi.LastModifiedAccessor;
import org.kitesdk.data.URIBuilder;
import org.kitesdk.data.spi.PartitionKey;
import org.kitesdk.data.spi.PartitionedDataset;
import org.kitesdk.data.user.NewUserRecord;

import static org.kitesdk.data.spi.filesystem.DatasetTestUtilities.USER_SCHEMA;
import static org.kitesdk.data.spi.filesystem.DatasetTestUtilities.checkTestUsers;
import static org.kitesdk.data.spi.filesystem.DatasetTestUtilities.datasetSize;
import static org.kitesdk.data.spi.filesystem.DatasetTestUtilities.writeTestUsers;
import java.io.IOException;
import java.net.URI;
import java.util.Arrays;
import java.util.Collection;

import static org.kitesdk.data.spi.filesystem.DatasetTestUtilities.*;

@RunWith(Parameterized.class)
public abstract class TestCrunchDatasets extends MiniDFSTest {
Expand Down Expand Up @@ -654,4 +632,92 @@ public void testMultipleFileReadingFromCrunch() throws IOException {

checkTestUsers(outputDataset, 10);
}

@Test
public void testMultipleFileReadingFromCrunchWithDifferentReaderWriterSchemas() {
Schema userNameOnlySchema = SchemaBuilder.record("userNameOnlyRecord")
.fields()
.requiredString("username")
.endRecord();

Schema emailOnlySchema = SchemaBuilder.record("emailOnlyRecord")
.fields()
.requiredString("email")
.endRecord();

// write two files, each of 5 records, using the original schema (username and email)
Dataset<GenericData.Record> writeDatasetA = repo.create("ns", "inA", new DatasetDescriptor.Builder()
.schema(USER_SCHEMA).build());
Dataset<GenericData.Record> writeDatasetB = repo.create("ns", "inB", new DatasetDescriptor.Builder()
.schema(USER_SCHEMA).build());
writeTestUsers(writeDatasetA, 5, 0);
writeTestUsers(writeDatasetB, 5, 5);

// update the schema of the repositories (using a schema with only the username or email field)
repo.update("ns", "inA", new DatasetDescriptor.Builder(repo.load("ns", "inA").getDescriptor())
.schema(userNameOnlySchema).build());
repo.update("ns", "inB", new DatasetDescriptor.Builder(repo.load("ns", "inB").getDescriptor())
.schema(emailOnlySchema).build());

// run a crunch singleInputPipeline to read/write the records using the reduced schemas
Dataset<GenericData.Record> inputA = repo.load("ns", "inA");
Dataset<GenericData.Record> inputB = repo.load("ns", "inB");

Dataset<GenericData.Record> outputDataset = repo.create("ns", "out", new DatasetDescriptor.Builder()
.schema(userNameOnlySchema).build());

Pipeline pipeline = new MRPipeline(TestCrunchDatasets.class);
PCollection<GenericData.Record> dataA = pipeline.read(CrunchDatasets.asSource(inputA))
.filter("remove records that don't have the correct schema",
new FilterRecordsWithExpectedSchemaFn(userNameOnlySchema.toString()));
PCollection<GenericData.Record> dataB = pipeline.read(CrunchDatasets.asSource(inputB))
.filter("remove records that don't have the correct schema",
new FilterRecordsWithExpectedSchemaFn(emailOnlySchema.toString()));
pipeline.write(dataA.union(dataB), CrunchDatasets.asTarget(outputDataset), Target.WriteMode.APPEND);
pipeline.run();

// If the records did not have the correct schema, they would have been filtered. So this checks that they all had the
// expected schema indeed.
checkReaderIteration(outputDataset.newReader(), 10, new NopRecordValidator());

// Repeat the same test with only a single input, to ensure that the simple case also works
Dataset<GenericData.Record> singleInputOutputDataset = repo.create("ns", "out2", new DatasetDescriptor.Builder()
.schema(userNameOnlySchema).build());

Pipeline singleInputPipeline = new MRPipeline(TestCrunchDatasets.class);
PCollection<GenericData.Record> singleInputFiltered = singleInputPipeline.read(CrunchDatasets.asSource(inputA))
.filter("remove records that don't have the correct schema",
new FilterRecordsWithExpectedSchemaFn(userNameOnlySchema.toString()));
singleInputPipeline.write(singleInputFiltered, CrunchDatasets.asTarget(singleInputOutputDataset), Target.WriteMode.APPEND);
singleInputPipeline.run();

checkReaderIteration(singleInputOutputDataset.newReader(), 5, new NopRecordValidator());
}

private static final class FilterRecordsWithExpectedSchemaFn extends FilterFn<Record> {

private final String expectedSchemaString;
private transient Schema expectedSchema;

private FilterRecordsWithExpectedSchemaFn(String expectedSchemaString) {
this.expectedSchemaString = expectedSchemaString;
}

@Override
public void initialize() {
this.expectedSchema = new Schema.Parser().parse(expectedSchemaString);
}

@Override
public boolean accept(GenericData.Record record) {
return expectedSchema.equals(record.getSchema());
}
}

private static class NopRecordValidator implements RecordValidator<Record> {
@Override
public void validate(Record record, int recordNum) {
// nop
}
}
}
Loading

0 comments on commit 46068dc

Please sign in to comment.