Skip to content

Commit 0130542

Browse files
author
Minh Nguyen
committed
Fix spotbug
1 parent 54a66c5 commit 0130542

File tree

8 files changed

+100
-79
lines changed

8 files changed

+100
-79
lines changed

internal/venice-common/src/main/java/com/linkedin/venice/serialization/avro/AvroProtocolDefinition.java

Lines changed: 2 additions & 17 deletions
Original file line numberDiff line numberDiff line change
@@ -235,7 +235,7 @@ private static Set<Byte> validateMagicBytes() {
235235
*/
236236
final String className;
237237

238-
Schema schema;
238+
final Schema schema;
239239

240240
/**
241241
* Indicates whether the protocol version should prepend the payload, or whether
@@ -290,7 +290,6 @@ private static Set<Byte> validateMagicBytes() {
290290
/**
291291
* Constructor for protocols where there is no SpecificRecord class, because the
292292
* schema defines only a primitive type, such as byte[].
293-
* SpecificRecord class can be set through setClass() method later.
294293
*/
295294
AvroProtocolDefinition(int currentProtocolVersion, Schema schema, String name) {
296295
this.magicByte = Optional.empty();
@@ -300,25 +299,11 @@ private static Set<Byte> validateMagicBytes() {
300299
this.schema = schema;
301300
}
302301

303-
/**
304-
* Set the SpecificRecord class for this protocol.
305-
* This method should be used only when the constructor without SpecificRecord class is used.
306-
*
307-
*/
308-
public void setClass(Class<? extends SpecificRecord> specificRecordClass) {
309-
if (!this.className.equals(specificRecordClass.getSimpleName())) {
310-
throw new VeniceException(
311-
"SpecificRecord class is already set to " + this.className + ", cannot set it to "
312-
+ specificRecordClass.getSimpleName());
313-
}
314-
this.schema = SpecificData.get().getSchema(specificRecordClass);
315-
}
316-
317302
public <T extends SpecificRecord> InternalAvroSpecificSerializer<T> getSerializer() {
318303
if (magicByte.isPresent() || protocolVersionStoredInHeader) {
319304
return new InternalAvroSpecificSerializer<>(this);
320305
}
321-
return new InternalAvroSpecificSerializer<>(this, 0);
306+
return new InternalAvroSpecificSerializer<>(this, 0, this.getCurrentProtocolVersionSchema());
322307
}
323308

324309
public int getCurrentProtocolVersion() {

internal/venice-common/src/main/java/com/linkedin/venice/serialization/avro/InternalAvroSpecificSerializer.java

Lines changed: 13 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -104,18 +104,26 @@ private static class ReusableObjects {
104104

105105
private final BiConsumer<Integer, Schema> newSchemaEncountered;
106106

107+
protected InternalAvroSpecificSerializer(
108+
AvroProtocolDefinition protocolDef,
109+
Integer payloadOffsetOverride,
110+
Schema compiledProtocolSchema) {
111+
this(protocolDef, payloadOffsetOverride, (schemaId, schema) -> {}, compiledProtocolSchema);
112+
}
113+
107114
protected InternalAvroSpecificSerializer(AvroProtocolDefinition protocolDef) {
108115
this(protocolDef, null);
109116
}
110117

111118
protected InternalAvroSpecificSerializer(AvroProtocolDefinition protocolDef, Integer payloadOffsetOverride) {
112-
this(protocolDef, payloadOffsetOverride, (schemaId, schema) -> {});
119+
this(protocolDef, payloadOffsetOverride, (schemaId, schema) -> {}, protocolDef.getCurrentProtocolVersionSchema());
113120
}
114121

115122
protected InternalAvroSpecificSerializer(
116123
AvroProtocolDefinition protocolDef,
117124
Integer payloadOffsetOverride,
118-
BiConsumer<Integer, Schema> newSchemaEncountered) {
125+
BiConsumer<Integer, Schema> newSchemaEncountered,
126+
Schema compiledProtocolSchema) {
119127
// Magic byte handling
120128
if (protocolDef.getMagicByte().isPresent()) {
121129
this.magicByte = protocolDef.getMagicByte().get();
@@ -125,6 +133,8 @@ protected InternalAvroSpecificSerializer(
125133
this.MAGIC_BYTE_LENGTH = 0;
126134
}
127135

136+
this.compiledProtocol = compiledProtocolSchema;
137+
128138
// Protocol version handling
129139
this.PROTOCOL_VERSION_OFFSET = MAGIC_BYTE_OFFSET + MAGIC_BYTE_LENGTH;
130140
if (protocolDef.protocolVersionStoredInHeader) {
@@ -156,9 +166,8 @@ protected InternalAvroSpecificSerializer(
156166
}
157167
this.PAYLOAD_OFFSET = payloadOffsetOverride;
158168
}
159-
this.compiledProtocol = protocolDef.getCurrentProtocolVersionSchema();
160169

161-
Map<Integer, Schema> protocolSchemaMap = Utils.getAllSchemasFromResources(protocolDef);
170+
Map<Integer, Schema> protocolSchemaMap = Utils.getAllSchemasFromResources(protocolDef, compiledProtocolSchema);
162171
int minimumSchemaId = protocolSchemaMap.keySet()
163172
.stream()
164173
.min(Integer::compareTo)

internal/venice-common/src/main/java/com/linkedin/venice/serialization/avro/KafkaValueSerializer.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -20,6 +20,6 @@ public KafkaValueSerializer() {
2020
}
2121

2222
public KafkaValueSerializer(BiConsumer<Integer, Schema> newSchemaEncountered) {
23-
super(AvroProtocolDefinition.KAFKA_MESSAGE_ENVELOPE, null, newSchemaEncountered);
23+
super(AvroProtocolDefinition.KAFKA_MESSAGE_ENVELOPE, null, newSchemaEncountered, null);
2424
}
2525
}

internal/venice-common/src/main/java/com/linkedin/venice/utils/Utils.java

Lines changed: 8 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -442,6 +442,12 @@ public static Schema getSchemaFromResource(String resourcePath) throws IOExcepti
442442
}
443443

444444
public static Map<Integer, Schema> getAllSchemasFromResources(AvroProtocolDefinition protocolDef) {
445+
return getAllSchemasFromResources(protocolDef, protocolDef.getCurrentProtocolVersionSchema());
446+
}
447+
448+
public static Map<Integer, Schema> getAllSchemasFromResources(
449+
AvroProtocolDefinition protocolDef,
450+
Schema compiledSchema) {
445451
final int SENTINEL_PROTOCOL_VERSION_USED_FOR_UNDETECTABLE_COMPILED_SCHEMA =
446452
InternalAvroSpecificSerializer.SENTINEL_PROTOCOL_VERSION_USED_FOR_UNDETECTABLE_COMPILED_SCHEMA;
447453
final int SENTINEL_PROTOCOL_VERSION_USED_FOR_UNVERSIONED_PROTOCOL =
@@ -480,7 +486,7 @@ public static Map<Integer, Schema> getAllSchemasFromResources(AvroProtocolDefini
480486
try {
481487
Schema schema = Utils.getSchemaFromResource(versionPath);
482488
protocolSchemaMap.put(version, schema);
483-
if (schema.equals(protocolDef.getCurrentProtocolVersionSchema())) {
489+
if (schema.equals(compiledSchema)) {
484490
compiledProtocolVersion = (byte) version;
485491
break;
486492
}
@@ -523,7 +529,7 @@ public static Map<Integer, Schema> getAllSchemasFromResources(AvroProtocolDefini
523529
if (intendedCurrentProtocol == null) {
524530
throw new VeniceException(
525531
"Failed to get schema for current version: " + currentProtocolVersion + " class: " + className);
526-
} else if (!intendedCurrentProtocol.equals(protocolDef.getCurrentProtocolVersionSchema())) {
532+
} else if (!intendedCurrentProtocol.equals(compiledSchema)) {
527533
throw new VeniceException(
528534
"The intended protocol version (" + currentProtocolVersion
529535
+ ") does not match the compiled protocol version (" + compiledProtocolVersion + ").");
Lines changed: 0 additions & 21 deletions
Original file line numberDiff line numberDiff line change
@@ -1,12 +1,8 @@
11
package com.linkedin.venice.serialization.avro;
22

3-
import static com.linkedin.venice.serialization.avro.AvroProtocolDefinition.ADMIN_OPERATION;
43
import static com.linkedin.venice.serialization.avro.AvroProtocolDefinition.KAFKA_MESSAGE_ENVELOPE;
54
import static com.linkedin.venice.serialization.avro.AvroProtocolDefinition.SERVER_ADMIN_RESPONSE;
65

7-
import com.linkedin.venice.exceptions.VeniceException;
8-
import com.linkedin.venice.kafka.protocol.KafkaMessageEnvelope;
9-
import org.apache.avro.specific.SpecificRecord;
106
import org.testng.Assert;
117
import org.testng.annotations.Test;
128

@@ -17,21 +13,4 @@ public void testGetSerializer() {
1713
Assert.assertNotNull(KAFKA_MESSAGE_ENVELOPE.getSerializer());
1814
Assert.assertNotNull(SERVER_ADMIN_RESPONSE.getSerializer());
1915
}
20-
21-
@Test
22-
public void testSetClass() {
23-
// No error should be thrown when setting the correct class
24-
Class<? extends SpecificRecord> KafkaMessageEnvelopeClass = KafkaMessageEnvelope.class;
25-
KAFKA_MESSAGE_ENVELOPE.setClass(KafkaMessageEnvelopeClass);
26-
27-
// An error should be thrown when setting an incorrect class
28-
try {
29-
ADMIN_OPERATION.setClass(KafkaMessageEnvelopeClass);
30-
Assert.fail("Expected an IllegalArgumentException to be thrown");
31-
} catch (VeniceException e) {
32-
Assert.assertEquals(
33-
e.getMessage(),
34-
"SpecificRecord class is already set to AdminOperation, cannot set it to KafkaMessageEnvelope");
35-
}
36-
}
3716
}

internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/controller/kafka/consumer/AdminConsumptionWithProtocolRollbackIntegrationTest.java

Lines changed: 46 additions & 26 deletions
Original file line numberDiff line numberDiff line change
@@ -1,7 +1,10 @@
11
package com.linkedin.venice.controller.kafka.consumer;
22

33
import static com.linkedin.venice.ConfigKeys.*;
4+
import static com.linkedin.venice.utils.AvroSchemaUtils.*;
45

6+
import com.linkedin.avroutil1.compatibility.AvroCompatibilityHelper;
7+
import com.linkedin.avroutil1.compatibility.FieldBuilder;
58
import com.linkedin.venice.controller.VeniceParentHelixAdmin;
69
import com.linkedin.venice.controller.kafka.AdminTopicUtils;
710
import com.linkedin.venice.controller.kafka.protocol.admin.AdminOperation;
@@ -35,7 +38,6 @@
3538
import java.util.ArrayList;
3639
import java.util.Collections;
3740
import java.util.List;
38-
import java.util.Map;
3941
import java.util.Properties;
4042
import java.util.concurrent.TimeUnit;
4143
import org.apache.avro.Schema;
@@ -61,15 +63,14 @@
6163
public class AdminConsumptionWithProtocolRollbackIntegrationTest {
6264
private static final int TIMEOUT = 1 * Time.MS_PER_MINUTE;
6365

64-
private final AdminOperationSerializer adminOperationSerializer = new AdminOperationSerializer();
66+
private AdminOperationSerializer adminOperationSerializer;
6567

6668
private static final String owner = "test_owner";
6769
private static final String keySchema = "\"string\"";
6870
private static final String valueSchema = "\"string\"";
6971
private static final int adminConsumptionMaxWorkerPoolSize = 3;
7072

7173
private VeniceTwoLayerMultiRegionMultiClusterWrapper venice;
72-
private AdminConsumerService adminConsumerService;
7374
private ControllerClient parentControllerClient;
7475
private VeniceParentHelixAdmin parentHelixAdmin;
7576
private VeniceWriter<byte[], byte[], byte[]> writer;
@@ -102,7 +103,6 @@ public void setUp() {
102103
VeniceControllerWrapper parentController = venice.getParentControllers().get(0);
103104
parentControllerClient = new ControllerClient(venice.getClusterNames()[0], parentController.getControllerUrl());
104105
clusterName = venice.getClusterNames()[0];
105-
adminConsumerService = parentController.getAdminConsumerServiceByCluster(clusterName);
106106
parentHelixAdmin = (VeniceParentHelixAdmin) parentController.getVeniceAdmin();
107107
PubSubTopicRepository pubSubTopicRepository = parentHelixAdmin.getPubSubTopicRepository();
108108
TopicManager topicManager = parentHelixAdmin.getTopicManager();
@@ -114,6 +114,7 @@ public void setUp() {
114114
pubSubBrokerWrapper.getPubSubClientsFactory().getProducerAdapterFactory();
115115
writer = IntegrationTestPushUtils.getVeniceWriterFactory(pubSubBrokerWrapper, pubSubProducerAdapterFactory)
116116
.createVeniceWriter(new VeniceWriterOptions.Builder(adminTopic.getName()).build());
117+
adminOperationSerializer = parentHelixAdmin.getAdminOperationSerializer();
117118
}
118119

119120
@AfterClass
@@ -193,51 +194,48 @@ private void addSchemaToSchemaSystemStore(int schemaId, Schema schema) {
193194
* and register it to schema system store with new schema id
194195
*/
195196
private int createNewSchemaAndRegister(int originalSchemaId) {
196-
Schema schema = AdminOperationSerializer.getSchema(originalSchemaId);
197+
Schema schema = adminOperationSerializer.getSchema(originalSchemaId);
197198

198199
int newSchemaId = originalSchemaId + 1;
199200
Schema newSchema = createNewSchema(schema);
200201
// We use the same schema and by-pass compatibility check for testing purpose
201202
addSchemaToSchemaSystemStore(newSchemaId, newSchema);
202203

203-
AdminOperationSerializer adminOperationSerializer = parentHelixAdmin.getAdminOperationSerializer();
204204
adminOperationSerializer.addSchema(newSchemaId, newSchema);
205205
return newSchemaId;
206206
}
207207

208208
/**
209209
* Create a new schema by adding a new field to the original schema
210210
*/
211-
private Schema createNewSchema(Schema originalSchema) {
212-
// Convert schema to string
213-
Schema newSchema = Schema.createRecord(
211+
private Schema createNewSchema(final Schema originalSchema) {
212+
if (originalSchema == null) {
213+
throw new IllegalArgumentException("Original schema cannot be null");
214+
}
215+
216+
// Create a new record schema with defensive copies
217+
final Schema newSchema = Schema.createRecord(
214218
originalSchema.getName(),
215219
originalSchema.getDoc(),
216220
originalSchema.getNamespace(),
217221
originalSchema.isError());
218222

219-
// Create a deep copy of all fields
220-
List<Schema.Field> newFields = new ArrayList<>();
223+
// Deep copy fields
224+
final List<Schema.Field> newFields = new ArrayList<>();
221225
for (Schema.Field field: originalSchema.getFields()) {
222-
Schema.Field newField = new Schema.Field(
223-
field.name(),
224-
field.schema(), // This creates a new field with the same schema
225-
field.doc(),
226-
field.defaultVal());
227-
// Copy all properties
228-
for (Map.Entry<String, Object> prop: field.getObjectProps().entrySet()) {
229-
newField.addProp(prop.getKey(), prop.getValue());
230-
}
231-
newFields.add(newField);
226+
// Defensive copy of schema (Avro schemas are immutable, so this is safe)
227+
FieldBuilder newField = deepCopySchemaFieldWithoutFieldProps(field);
228+
copyFieldProperties(newField, field);
229+
newFields.add(newField.build());
232230
}
233231

234-
// Add the new field
235-
Schema.Field testField =
236-
new Schema.Field("testField", Schema.create(Schema.Type.INT), "Documentation for testField", 0);
232+
// Add extra field
233+
final Schema.Field testField = AvroCompatibilityHelper
234+
.createSchemaField("testField", Schema.create(Schema.Type.INT), "Documentation for testField", 0);
237235
newFields.add(testField);
238236

239-
// Set all fields at once
240-
newSchema.setFields(newFields);
237+
// Set fields using defensive copy
238+
newSchema.setFields(Collections.unmodifiableList(new ArrayList<>(newFields)));
241239

242240
return newSchema;
243241
}
@@ -302,4 +300,26 @@ private byte[] getStoreUpdateMessage(
302300
private int nextExecutionId() {
303301
return executionId++;
304302
}
303+
304+
private static FieldBuilder deepCopySchemaFieldWithoutFieldProps(Schema.Field field) {
305+
FieldBuilder fieldBuilder = AvroCompatibilityHelper.newField(null)
306+
.setName(field.name())
307+
.setSchema(field.schema())
308+
.setDoc(field.doc())
309+
.setOrder(field.order());
310+
// set default as AvroCompatibilityHelper builder might drop defaults if there is type mismatch
311+
if (field.hasDefaultValue()) {
312+
fieldBuilder.setDefault(getFieldDefault(field));
313+
}
314+
return fieldBuilder;
315+
}
316+
317+
private static void copyFieldProperties(FieldBuilder fieldBuilder, Schema.Field field) {
318+
AvroCompatibilityHelper.getAllPropNames(field).forEach(k -> {
319+
String propValue = AvroCompatibilityHelper.getFieldPropAsJsonString(field, k);
320+
if (propValue != null) {
321+
fieldBuilder.addProp(k, propValue);
322+
}
323+
});
324+
}
305325
}

services/venice-controller/src/main/java/com/linkedin/venice/controller/init/SystemSchemaInitializationRoutine.java

Lines changed: 6 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -20,6 +20,7 @@
2020
import java.util.Map;
2121
import java.util.Optional;
2222
import org.apache.avro.Schema;
23+
import org.apache.avro.specific.SpecificData;
2324
import org.apache.avro.specific.SpecificRecord;
2425
import org.apache.logging.log4j.LogManager;
2526
import org.apache.logging.log4j.Logger;
@@ -30,6 +31,7 @@ public class SystemSchemaInitializationRoutine implements ClusterLeaderInitializ
3031
private static final String DEFAULT_KEY_SCHEMA_STR = "\"int\"";
3132

3233
private final AvroProtocolDefinition protocolDefinition;
34+
private final Schema protocolSchema;
3335
private final VeniceControllerMultiClusterConfig multiClusterConfigs;
3436
private final VeniceHelixAdmin admin;
3537
private final Optional<Schema> keySchema;
@@ -69,16 +71,14 @@ public SystemSchemaInitializationRoutine(
6971
boolean autoRegisterDerivedComputeSchema,
7072
Class<? extends SpecificRecord> specificRecordClass) {
7173
this.protocolDefinition = protocolDefinition;
74+
this.protocolSchema = specificRecordClass != null
75+
? SpecificData.get().getSchema(specificRecordClass)
76+
: protocolDefinition.getCurrentProtocolVersionSchema();
7277
this.multiClusterConfigs = multiClusterConfigs;
7378
this.admin = admin;
7479
this.keySchema = keySchema;
7580
this.storeMetadataUpdate = storeMetadataUpdate;
7681
this.autoRegisterDerivedComputeSchema = autoRegisterDerivedComputeSchema;
77-
// Set the specific record class if provided. This allows us to override class and schema for protocol definitions
78-
// that is in venice-controller and cannot reference in venice-common.
79-
if (specificRecordClass != null) {
80-
protocolDefinition.setClass(specificRecordClass);
81-
}
8282
}
8383

8484
/**
@@ -89,7 +89,7 @@ public void execute(String clusterToInit) {
8989
String intendedCluster = multiClusterConfigs.getSystemSchemaClusterName();
9090
if (intendedCluster.equals(clusterToInit)) {
9191
String systemStoreName = protocolDefinition.getSystemStoreName();
92-
Map<Integer, Schema> protocolSchemaMap = Utils.getAllSchemasFromResources(protocolDefinition);
92+
Map<Integer, Schema> protocolSchemaMap = Utils.getAllSchemasFromResources(protocolDefinition, protocolSchema);
9393

9494
// Sanity check to make sure the store is not already created in another cluster.
9595
try {

0 commit comments

Comments
 (0)