-
Notifications
You must be signed in to change notification settings - Fork 1.1k
/
Copy pathAvroConverter.java
202 lines (177 loc) · 6.98 KB
/
AvroConverter.java
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
/*
* Copyright 2018 Confluent 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 io.confluent.connect.avro;
import io.confluent.kafka.schemaregistry.avro.AvroSchema;
import io.confluent.kafka.schemaregistry.avro.AvroSchemaProvider;
import io.confluent.kafka.schemaregistry.client.SchemaRegistryClient;
import io.confluent.kafka.schemaregistry.client.SchemaRegistryClientFactory;
import io.confluent.kafka.serializers.AbstractKafkaAvroDeserializer;
import io.confluent.kafka.serializers.AbstractKafkaAvroSerializer;
import io.confluent.kafka.serializers.GenericContainerWithVersion;
import io.confluent.kafka.serializers.KafkaAvroDeserializerConfig;
import io.confluent.kafka.serializers.KafkaAvroSerializerConfig;
import io.confluent.kafka.serializers.NonRecordContainer;
import org.apache.avro.generic.GenericContainer;
import org.apache.avro.generic.IndexedRecord;
import org.apache.kafka.common.config.ConfigException;
import org.apache.kafka.common.errors.InvalidConfigurationException;
import org.apache.kafka.common.errors.SerializationException;
import org.apache.kafka.common.errors.TimeoutException;
import org.apache.kafka.common.header.Headers;
import org.apache.kafka.connect.data.Schema;
import org.apache.kafka.connect.data.SchemaAndValue;
import org.apache.kafka.connect.errors.DataException;
import org.apache.kafka.connect.errors.RetriableException;
import org.apache.kafka.connect.storage.Converter;
import java.util.Collections;
import java.util.Map;
/**
* Implementation of Converter that uses Avro schemas and objects.
*/
public class AvroConverter implements Converter {
private SchemaRegistryClient schemaRegistry;
private Serializer serializer;
private Deserializer deserializer;
private boolean isKey;
private AvroData avroData;
public AvroConverter() {
}
// Public only for testing
public AvroConverter(SchemaRegistryClient client) {
schemaRegistry = client;
}
@Override
public void configure(Map<String, ?> configs, boolean isKey) {
this.isKey = isKey;
AvroConverterConfig avroConverterConfig = new AvroConverterConfig(configs);
if (schemaRegistry == null) {
schemaRegistry = SchemaRegistryClientFactory.newClient(
avroConverterConfig.getSchemaRegistryUrls(),
avroConverterConfig.getMaxSchemasPerSubject(),
Collections.singletonList(new AvroSchemaProvider()),
configs,
avroConverterConfig.requestHeaders()
);
}
serializer = new Serializer(configs, schemaRegistry);
deserializer = new Deserializer(configs, schemaRegistry);
avroData = new AvroData(new AvroDataConfig(configs));
}
@Override
public byte[] fromConnectData(String topic, Schema schema, Object value) {
return fromConnectData(topic, null, schema, value);
}
@Override
public byte[] fromConnectData(String topic, Headers headers, Schema schema, Object value) {
try {
org.apache.avro.Schema avroSchema = avroData.fromConnectSchema(schema);
return serializer.serialize(
topic,
isKey,
headers,
avroData.fromConnectData(schema, avroSchema, value),
new AvroSchema(avroSchema));
} catch (TimeoutException e) {
throw new RetriableException(
String.format("Failed to serialize Avro data from topic %s :", topic),
e
);
} catch (SerializationException e) {
throw new DataException(
String.format("Failed to serialize Avro data from topic %s :", topic),
e
);
} catch (InvalidConfigurationException e) {
throw new ConfigException(
String.format("Failed to access Avro data from topic %s : %s", topic, e.getMessage())
);
}
}
@Override
public SchemaAndValue toConnectData(String topic, byte[] value) {
return toConnectData(topic, null, value);
}
@Override
public SchemaAndValue toConnectData(String topic, Headers headers, byte[] value) {
try {
GenericContainerWithVersion containerWithVersion =
deserializer.deserialize(topic, isKey, headers, value);
if (containerWithVersion == null) {
return SchemaAndValue.NULL;
}
GenericContainer deserialized = containerWithVersion.container();
Integer version = containerWithVersion.version();
if (deserialized instanceof IndexedRecord) {
return avroData.toConnectData(deserialized.getSchema(), deserialized, version);
} else if (deserialized instanceof NonRecordContainer) {
return avroData.toConnectData(
deserialized.getSchema(), ((NonRecordContainer) deserialized).getValue(), version);
}
throw new DataException(
String.format("Unsupported type returned during deserialization of topic %s ", topic)
);
} catch (TimeoutException e) {
throw new RetriableException(
String.format("Failed to deserialize data for topic %s to Avro: ", topic),
e
);
} catch (SerializationException e) {
throw new DataException(
String.format("Failed to deserialize data for topic %s to Avro: ", topic),
e
);
} catch (InvalidConfigurationException e) {
throw new ConfigException(
String.format("Failed to access Avro data from topic %s : %s", topic, e.getMessage())
);
}
}
private static class Serializer extends AbstractKafkaAvroSerializer {
public Serializer(SchemaRegistryClient client, boolean autoRegisterSchema) {
schemaRegistry = client;
this.autoRegisterSchema = autoRegisterSchema;
}
public Serializer(Map<String, ?> configs, SchemaRegistryClient client) {
this(client, false);
configure(new KafkaAvroSerializerConfig(configs));
}
public byte[] serialize(
String topic, boolean isKey, Headers headers, Object value, AvroSchema schema) {
if (value == null) {
return null;
}
return serializeImpl(
getSubjectName(topic, isKey, value, schema),
topic,
headers,
value,
schema);
}
}
private static class Deserializer extends AbstractKafkaAvroDeserializer {
public Deserializer(SchemaRegistryClient client) {
schemaRegistry = client;
}
public Deserializer(Map<String, ?> configs, SchemaRegistryClient client) {
this(client);
configure(new KafkaAvroDeserializerConfig(configs));
}
public GenericContainerWithVersion deserialize(
String topic, boolean isKey, Headers headers, byte[] payload) {
return deserializeWithSchemaAndVersion(topic, isKey, headers, payload);
}
}
}