-
Notifications
You must be signed in to change notification settings - Fork 1.1k
/
AbstractKafkaSchemaSerDe.java
281 lines (246 loc) · 9.96 KB
/
AbstractKafkaSchemaSerDe.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
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
/*
* 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.kafka.serializers;
import io.confluent.kafka.schemaregistry.client.SchemaMetadata;
import io.confluent.kafka.schemaregistry.utils.BoundedConcurrentHashMap;
import java.util.Objects;
import java.util.Optional;
import io.confluent.kafka.schemaregistry.utils.QualifiedSubject;
import io.confluent.kafka.serializers.context.NullContextNameStrategy;
import io.confluent.kafka.serializers.context.strategy.ContextNameStrategy;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericContainer;
import org.apache.kafka.common.KafkaException;
import org.apache.kafka.common.errors.InvalidConfigurationException;
import org.apache.kafka.common.errors.SerializationException;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.util.Collections;
import java.util.List;
import java.util.Map;
import io.confluent.kafka.schemaregistry.ParsedSchema;
import io.confluent.kafka.schemaregistry.SchemaProvider;
import io.confluent.kafka.schemaregistry.client.CachedSchemaRegistryClient;
import io.confluent.kafka.schemaregistry.client.SchemaRegistryClient;
import io.confluent.kafka.schemaregistry.client.rest.exceptions.RestClientException;
import io.confluent.kafka.schemaregistry.testutil.MockSchemaRegistry;
import io.confluent.kafka.serializers.subject.strategy.SubjectNameStrategy;
import io.confluent.kafka.serializers.subject.TopicNameStrategy;
import javax.ws.rs.core.Response.Status.Family;
/**
* Common fields and helper methods for both the serializer and the deserializer.
*/
public abstract class AbstractKafkaSchemaSerDe {
protected static final byte MAGIC_BYTE = 0x0;
protected static final int idSize = 4;
private static int DEFAULT_CACHE_CAPACITY = 1000;
protected SchemaRegistryClient schemaRegistry;
protected ContextNameStrategy contextNameStrategy = new NullContextNameStrategy();
protected Object keySubjectNameStrategy = new TopicNameStrategy();
protected Object valueSubjectNameStrategy = new TopicNameStrategy();
protected Map<SubjectSchema, ParsedSchema> latestVersions =
new BoundedConcurrentHashMap<>(DEFAULT_CACHE_CAPACITY);
protected boolean useSchemaReflection;
protected void configureClientProperties(
AbstractKafkaSchemaSerDeConfig config,
SchemaProvider provider) {
List<String> urls = config.getSchemaRegistryUrls();
int maxSchemaObject = config.getMaxSchemasPerSubject();
Map<String, Object> originals = config.originalsWithPrefix("");
if (null == schemaRegistry) {
String mockScope = MockSchemaRegistry.validateAndMaybeGetMockScope(urls);
List<SchemaProvider> providers = Collections.singletonList(provider);
if (mockScope != null) {
schemaRegistry = MockSchemaRegistry.getClientForScope(mockScope, providers);
} else {
schemaRegistry = new CachedSchemaRegistryClient(
urls,
maxSchemaObject,
providers,
originals,
config.requestHeaders()
);
}
}
contextNameStrategy = config.contextNameStrategy();
keySubjectNameStrategy = config.keySubjectNameStrategy();
valueSubjectNameStrategy = config.valueSubjectNameStrategy();
useSchemaReflection = config.useSchemaReflection();
}
/**
* Get the subject name for the given topic and value type.
*/
protected String getSubjectName(String topic, boolean isKey, Object value, ParsedSchema schema) {
Object subjectNameStrategy = subjectNameStrategy(isKey);
String subject;
if (subjectNameStrategy instanceof SubjectNameStrategy) {
subject = ((SubjectNameStrategy) subjectNameStrategy).subjectName(topic, isKey, schema);
} else {
subject = ((io.confluent.kafka.serializers.subject.SubjectNameStrategy) subjectNameStrategy)
.getSubjectName(topic, isKey, value);
}
return getContextName(topic, subject);
}
protected String getContextName(String topic) {
return getContextName(topic, null);
}
private String getContextName(String topic, String subject) {
String contextName = contextNameStrategy.contextName(topic);
if (contextName != null) {
QualifiedSubject cs = new QualifiedSubject(null, contextName, subject);
return cs.toQualifiedSubject();
} else {
return subject;
}
}
protected boolean strategyUsesSchema(boolean isKey) {
Object subjectNameStrategy = subjectNameStrategy(isKey);
if (subjectNameStrategy instanceof SubjectNameStrategy) {
return ((SubjectNameStrategy) subjectNameStrategy).usesSchema();
} else {
return false;
}
}
protected boolean isDeprecatedSubjectNameStrategy(boolean isKey) {
Object subjectNameStrategy = subjectNameStrategy(isKey);
return !(
subjectNameStrategy
instanceof io.confluent.kafka.serializers.subject.strategy.SubjectNameStrategy);
}
private Object subjectNameStrategy(boolean isKey) {
return isKey ? keySubjectNameStrategy : valueSubjectNameStrategy;
}
/**
* Get the subject name used by the old Encoder interface, which relies only on the value type
* rather than the topic.
*/
protected String getOldSubjectName(Object value) {
if (value instanceof GenericContainer) {
return ((GenericContainer) value).getSchema().getName() + "-value";
} else {
throw new SerializationException("Primitive types are not supported yet");
}
}
@Deprecated
public int register(String subject, Schema schema) throws IOException, RestClientException {
return schemaRegistry.register(subject, schema);
}
public int register(String subject, ParsedSchema schema) throws IOException, RestClientException {
return schemaRegistry.register(subject, schema);
}
@Deprecated
public Schema getById(int id) throws IOException, RestClientException {
return schemaRegistry.getById(id);
}
public ParsedSchema getSchemaById(int id) throws IOException, RestClientException {
return schemaRegistry.getSchemaById(id);
}
@Deprecated
public Schema getBySubjectAndId(String subject, int id)
throws IOException, RestClientException {
return schemaRegistry.getBySubjectAndId(subject, id);
}
public ParsedSchema getSchemaBySubjectAndId(String subject, int id)
throws IOException, RestClientException {
return schemaRegistry.getSchemaBySubjectAndId(subject, id);
}
protected ParsedSchema lookupLatestVersion(
String subject, ParsedSchema schema, boolean latestCompatStrict)
throws IOException, RestClientException {
return lookupLatestVersion(schemaRegistry, subject, schema, latestVersions, latestCompatStrict);
}
protected static ParsedSchema lookupLatestVersion(
SchemaRegistryClient schemaRegistry,
String subject,
ParsedSchema schema,
Map<SubjectSchema, ParsedSchema> cache,
boolean latestCompatStrict)
throws IOException, RestClientException {
SubjectSchema ss = new SubjectSchema(subject, schema);
ParsedSchema latestVersion = null;
if (cache != null) {
latestVersion = cache.get(ss);
}
if (latestVersion == null) {
SchemaMetadata schemaMetadata = schemaRegistry.getLatestSchemaMetadata(subject);
Optional<ParsedSchema> optSchema =
schemaRegistry.parseSchema(
schemaMetadata.getSchemaType(),
schemaMetadata.getSchema(),
schemaMetadata.getReferences());
latestVersion = optSchema.orElseThrow(
() -> new IOException("Invalid schema " + schemaMetadata.getSchema()
+ " with refs " + schemaMetadata.getReferences()
+ " of type " + schemaMetadata.getSchemaType()));
// Sanity check by testing latest is backward compatibility with schema
// Don't test for forward compatibility so unions can be handled properly
if (latestCompatStrict && !latestVersion.isBackwardCompatible(schema).isEmpty()) {
throw new IOException("Incompatible schema " + schemaMetadata.getSchema()
+ " with refs " + schemaMetadata.getReferences()
+ " of type " + schemaMetadata.getSchemaType()
+ " for schema " + schema.canonicalString());
}
if (cache != null) {
cache.put(ss, latestVersion);
}
}
return latestVersion;
}
protected ByteBuffer getByteBuffer(byte[] payload) {
ByteBuffer buffer = ByteBuffer.wrap(payload);
if (buffer.get() != MAGIC_BYTE) {
throw new SerializationException("Unknown magic byte!");
}
return buffer;
}
protected static KafkaException toKafkaException(RestClientException e, String errorMessage) {
if (Family.familyOf(e.getErrorCode()) == Family.CLIENT_ERROR) {
return new InvalidConfigurationException(e.getMessage());
} else {
return new SerializationException(errorMessage, e);
}
}
protected static class SubjectSchema {
private String subject;
private ParsedSchema schema;
public SubjectSchema(String subject, ParsedSchema schema) {
this.subject = subject;
this.schema = schema;
}
public String getSubject() {
return subject;
}
public ParsedSchema getSchema() {
return schema;
}
@Override
public boolean equals(Object o) {
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
SubjectSchema that = (SubjectSchema) o;
return subject.equals(that.subject)
&& schema.equals(that.schema);
}
@Override
public int hashCode() {
return Objects.hash(subject, schema);
}
}
}