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

Added the ability to configure the topic-to-Avro-schema relationship manually (i.e. not using Confluent Schema Registry) #860

Merged
merged 1 commit into from
Aug 1, 2019
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
3 changes: 3 additions & 0 deletions pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -370,6 +370,9 @@
<testResource>
<directory>src/test/config</directory>
</testResource>
<testResource>
<directory>src/test/avro</directory>
</testResource>
</testResources>
<plugins>
<plugin>
Expand Down
28 changes: 28 additions & 0 deletions src/main/java/com/pinterest/secor/common/AvroSchemaRegistry.java
Original file line number Diff line number Diff line change
@@ -0,0 +1,28 @@
/*
* 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 com.pinterest.secor.common;

import org.apache.avro.Schema;
import org.apache.avro.generic.GenericRecord;

public interface AvroSchemaRegistry {
GenericRecord deserialize(String topic, byte[] payload);

Schema getSchema(String topic);
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,70 @@
/*
* 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 com.pinterest.secor.common;

import com.pinterest.secor.parser.AvroMessageParser;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericRecord;
import org.apache.avro.io.Decoder;
import org.apache.avro.io.DecoderFactory;
import org.apache.avro.specific.SpecificDatumReader;
import org.apache.kafka.common.errors.SerializationException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import java.io.IOException;
import java.util.HashMap;
import java.util.Map;
import java.util.stream.Collectors;

public class ConfigurableAvroSchemaRegistry implements AvroSchemaRegistry {
private static final Logger LOG = LoggerFactory.getLogger(AvroMessageParser.class);
private final Map<String, Schema> schemas = new HashMap<>();
private final Map<String, SpecificDatumReader<GenericRecord>> readers = new HashMap<>();

public ConfigurableAvroSchemaRegistry(SecorConfig config) {
schemas.putAll(config.getAvroMessageSchema().entrySet().stream().collect(Collectors.toMap(Map.Entry::getKey, e -> getAvroSchema(e.getValue()))));
readers.putAll(schemas.entrySet().stream().collect(Collectors.toMap(Map.Entry::getKey, e -> new SpecificDatumReader<>(e.getValue()))));
}

private Schema getAvroSchema(String path) {
Schema schema = null;
try {
schema = new Schema.Parser().parse(getClass().getResourceAsStream(path));
} catch (Exception ex) {
LOG.error("Exception getting schema for file " + path, ex);
}
return schema;
}

public GenericRecord deserialize(String topic, byte[] payload) {
GenericRecord record = null;
try {
Decoder decoder = DecoderFactory.get().binaryDecoder(payload, null);
record = readers.get(topic).read(null, decoder);
} catch (IOException ioe) {
throw new SerializationException("Error deserializing Avro message");
}
return record;
}

public Schema getSchema(String topic) {
return schemas.get(topic);
}
}
12 changes: 6 additions & 6 deletions src/main/java/com/pinterest/secor/common/SecorConfig.java
Original file line number Diff line number Diff line change
Expand Up @@ -26,11 +26,7 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import java.util.HashMap;
import java.util.Iterator;
import java.util.Map;
import java.util.Properties;
import java.util.TimeZone;
import java.util.*;

/**
* One-stop shop for Secor configuration options.
Expand Down Expand Up @@ -782,10 +778,14 @@ public Map<String, String> getPropertyMapForPrefix(String prefix) {
}
return map;
}

public Map<String, String> getORCMessageSchema() {
return getPropertyMapForPrefix("secor.orc.message.schema");
}

public Map<String, String> getAvroMessageSchema() {
return getPropertyMapForPrefix("secor.avro.message.schema");
}

public String getORCSchemaProviderClass(){
return getString("secor.orc.schema.provider");
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -30,7 +30,7 @@
import java.util.Properties;
import java.util.concurrent.ConcurrentHashMap;

public class SecorSchemaRegistryClient {
public class SecorSchemaRegistryClient implements AvroSchemaRegistry {

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

Expand All @@ -55,7 +55,7 @@ protected void init(SecorConfig config) {
deserializer = new KafkaAvroDeserializer(schemaRegistryClient);
}

public GenericRecord decodeMessage(String topic, byte[] message) {
public GenericRecord deserialize(String topic, byte[] message) {
if (message.length == 0) {
message = null;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,61 +18,48 @@
*/
package com.pinterest.secor.io.impl;

import java.io.ByteArrayOutputStream;
import java.io.File;
import java.io.IOException;
import java.nio.ByteBuffer;

import com.google.protobuf.Message;
import com.pinterest.secor.common.SecorSchemaRegistryClient;
import com.pinterest.secor.util.FileUtil;
import com.pinterest.secor.common.AvroSchemaRegistry;
import com.pinterest.secor.common.LogFilePath;
import com.pinterest.secor.common.SecorConfig;
import com.pinterest.secor.io.FileReader;
import com.pinterest.secor.io.FileReaderWriterFactory;
import com.pinterest.secor.io.FileWriter;
import com.pinterest.secor.io.KeyValue;
import com.pinterest.secor.util.AvroSchemaRegistryFactory;
import com.pinterest.secor.util.AvroSerializer;
import com.pinterest.secor.util.ParquetUtil;
import org.apache.avro.AvroRuntimeException;
import org.apache.avro.Schema;
import org.apache.avro.file.CodecFactory;
import org.apache.avro.file.DataFileReader;
import org.apache.avro.file.DataFileWriter;
import org.apache.avro.generic.GenericRecord;
import org.apache.avro.io.DatumReader;
import org.apache.avro.io.DatumWriter;
import org.apache.avro.io.Encoder;
import org.apache.avro.io.EncoderFactory;
import org.apache.avro.specific.SpecificDatumReader;
import org.apache.avro.specific.SpecificDatumWriter;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.compress.CompressionCodec;
import org.apache.hadoop.io.compress.CompressionCodecFactory;
import org.apache.parquet.avro.AvroParquetReader;
import org.apache.parquet.avro.AvroParquetWriter;
import org.apache.parquet.hadoop.ParquetReader;
import org.apache.parquet.hadoop.ParquetWriter;
import org.apache.parquet.hadoop.metadata.CompressionCodecName;

import com.pinterest.secor.common.LogFilePath;
import com.pinterest.secor.common.SecorConfig;
import com.pinterest.secor.io.FileReader;
import com.pinterest.secor.io.FileReaderWriterFactory;
import com.pinterest.secor.io.FileWriter;
import com.pinterest.secor.io.KeyValue;
import com.pinterest.secor.util.ParquetUtil;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import java.io.File;
import java.io.IOException;

public class AvroFileReaderWriterFactory implements FileReaderWriterFactory {

private static final Logger LOG = LoggerFactory.getLogger(AvroFileReaderWriterFactory.class);
protected final int blockSize;
protected final int pageSize;
protected final boolean enableDictionary;
protected final boolean validating;
protected SecorSchemaRegistryClient schemaRegistryClient;
protected AvroSchemaRegistry schemaRegistry;

public AvroFileReaderWriterFactory(SecorConfig config) {
blockSize = ParquetUtil.getParquetBlockSize(config);
pageSize = ParquetUtil.getParquetPageSize(config);
enableDictionary = ParquetUtil.getParquetEnableDictionary(config);
validating = ParquetUtil.getParquetValidation(config);
schemaRegistryClient = new SecorSchemaRegistryClient(config);
schemaRegistry = AvroSchemaRegistryFactory.getSchemaRegistry(config);
}

@Override
Expand All @@ -85,16 +72,6 @@ public FileWriter BuildFileWriter(LogFilePath logFilePath, CompressionCodec code
return new AvroFileWriter(logFilePath, codec);
}

protected static byte[] serializeAvroRecord(SpecificDatumWriter<GenericRecord> writer, GenericRecord record) throws IOException {
ByteArrayOutputStream out = new ByteArrayOutputStream();
Encoder encoder = EncoderFactory.get().directBinaryEncoder(out, null);
writer.write(record, encoder);
encoder.flush();
ByteBuffer serialized = ByteBuffer.allocate(out.toByteArray().length);
serialized.put(out.toByteArray());
return serialized.array();
}

protected class AvroFileReader implements FileReader {

private DataFileReader<GenericRecord> reader;
Expand All @@ -106,7 +83,7 @@ public AvroFileReader(LogFilePath logFilePath, CompressionCodec codec) throws IO
file = new File(logFilePath.getLogFilePath());
file.getParentFile().mkdirs();
String topic = logFilePath.getTopic();
Schema schema = schemaRegistryClient.getSchema(topic);
Schema schema = schemaRegistry.getSchema(topic);

DatumReader datumReader = new SpecificDatumReader(schema);
try {
Expand All @@ -123,7 +100,7 @@ public AvroFileReader(LogFilePath logFilePath, CompressionCodec codec) throws IO
public KeyValue next() throws IOException {
GenericRecord record = reader.next();
if (record != null) {
return new KeyValue(offset++, serializeAvroRecord(writer, record));
return new KeyValue(offset++, AvroSerializer.serialize(writer, record));
}
return null;
}
Expand All @@ -147,7 +124,7 @@ public AvroFileWriter(LogFilePath logFilePath, CompressionCodec codec) throws IO
file.getParentFile().mkdirs();
LOG.debug("Creating Brand new Writer for path {}", logFilePath.getLogFilePath());
topic = logFilePath.getTopic();
Schema schema = schemaRegistryClient.getSchema(topic);
Schema schema = schemaRegistry.getSchema(topic);
SpecificDatumWriter specificDatumWriter= new SpecificDatumWriter(schema);
writer = new DataFileWriter(specificDatumWriter);
writer.setCodec(getCodecFactory(codec));
Expand All @@ -172,7 +149,7 @@ public long getLength() throws IOException {

@Override
public void write(KeyValue keyValue) throws IOException {
GenericRecord record = schemaRegistryClient.decodeMessage(topic, keyValue.getValue());
GenericRecord record = schemaRegistry.deserialize(topic, keyValue.getValue());
LOG.trace("Writing record {}", record);
if (record != null){
writer.append(record);
Expand Down
Loading