-
Notifications
You must be signed in to change notification settings - Fork 1.1k
/
Copy pathAbstractKafkaAvroSerDe.java
140 lines (123 loc) · 5.08 KB
/
AbstractKafkaAvroSerDe.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
/*
* 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.common.config.ConfigException;
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.TopicNameStrategy;
import io.confluent.kafka.serializers.subject.strategy.SubjectNameStrategy;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericContainer;
import org.apache.kafka.common.errors.SerializationException;
import java.io.IOException;
import java.util.LinkedList;
import java.util.List;
import java.util.Map;
/**
* Common fields and helper methods for both the serializer and the deserializer.
*/
public abstract class AbstractKafkaAvroSerDe {
protected static final byte MAGIC_BYTE = 0x0;
protected static final int idSize = 4;
private static final String MOCK_URL_PREFIX = "mock://";
protected SchemaRegistryClient schemaRegistry;
protected Object keySubjectNameStrategy = new TopicNameStrategy();
protected Object valueSubjectNameStrategy = new TopicNameStrategy();
protected void configureClientProperties(AbstractKafkaAvroSerDeConfig config) {
List<String> urls = config.getSchemaRegistryUrls();
int maxSchemaObject = config.getMaxSchemasPerSubject();
Map<String, Object> originals = config.originalsWithPrefix("");
if (null == schemaRegistry) {
String mockScope = validateAndMaybeGetMockScope(urls);
if (mockScope != null) {
schemaRegistry = MockSchemaRegistry.getClientForScope(mockScope);
} else {
schemaRegistry = new CachedSchemaRegistryClient(
urls,
maxSchemaObject,
originals,
config.requestHeaders()
);
}
}
keySubjectNameStrategy = config.keySubjectNameStrategy();
valueSubjectNameStrategy = config.valueSubjectNameStrategy();
}
private static String validateAndMaybeGetMockScope(final List<String> urls) {
final List<String> mockScopes = new LinkedList<>();
for (final String url : urls) {
if (url.startsWith(MOCK_URL_PREFIX)) {
mockScopes.add(url.substring(MOCK_URL_PREFIX.length()));
}
}
if (mockScopes.isEmpty()) {
return null;
} else if (mockScopes.size() > 1) {
throw new ConfigException(
"Only one mock scope is permitted for 'schema.registry.url'. Got: " + urls
);
} else if (urls.size() > mockScopes.size()) {
throw new ConfigException(
"Cannot mix mock and real urls for 'schema.registry.url'. Got: " + urls
);
} else {
return mockScopes.get(0);
}
}
/**
* Get the subject name for the given topic and value type.
*/
protected String getSubjectName(String topic, boolean isKey, Object value, Schema schema) {
Object subjectNameStrategy = subjectNameStrategy(isKey);
if (subjectNameStrategy instanceof SubjectNameStrategy) {
return ((SubjectNameStrategy) subjectNameStrategy).subjectName(topic, isKey, schema);
} else {
return ((io.confluent.kafka.serializers.subject.SubjectNameStrategy) subjectNameStrategy)
.getSubjectName(topic, isKey, value);
}
}
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");
}
}
public int register(String subject, Schema schema) throws IOException, RestClientException {
return schemaRegistry.register(subject, schema);
}
public Schema getById(int id) throws IOException, RestClientException {
return schemaRegistry.getById(id);
}
public Schema getBySubjectAndId(String subject, int id) throws IOException, RestClientException {
return schemaRegistry.getBySubjectAndId(subject, id);
}
}