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

[controller][test] Message header triggered new KME schema registrati… #628

Merged
merged 8 commits into from
Sep 22, 2023
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
Original file line number Diff line number Diff line change
Expand Up @@ -369,6 +369,9 @@ private ConfigKeys() {
public static final String SYSTEM_SCHEMA_INITIALIZATION_AT_START_TIME_ENABLED =
"system.schema.initialization.at.start.time.enabled";

public static final String KME_REGISTRATION_FROM_MESSAGE_HEADER_ENABLED =
"kme.registration.from.message.header.enabled";

// Server specific configs
public static final String LISTENER_PORT = "listener.port";
public static final String GRPC_READ_SERVER_PORT = "grpc.read.server.port";
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -558,4 +558,10 @@ private void preCheckStoreCondition(String storeName) {
throw new VeniceNoStoreException(storeName);
}
}

// For testing purpose only.
void removeValueSchema(String storeName, int schemaId) {
preCheckStoreCondition(storeName);
accessor.removeValueSchema(storeName, String.valueOf(schemaId));
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -299,4 +299,9 @@ public void clear() {
readOnlyZKSharedSchemaRepository.clear();
readWriteRegularStoreSchemaRepository.clear();
}

// For testing purpose only.
public ReadWriteSchemaRepository getReadWriteRegularStoreSchemaRepository() {
return this.readWriteRegularStoreSchemaRepository;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -278,4 +278,10 @@ String getReplicationMetadataSchemaPath(String storeName, String replicationMeta
.append(replicationMetadataVersionIdPair)
.toString();
}

// For testing purpose only.
public void removeValueSchema(String storeName, String schemaId) {
HelixUtils.remove(schemaAccessor, getValueSchemaPath(storeName, schemaId));
logger.info("Removed value schema: {} for store: {}.", schemaId, storeName);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -65,7 +65,10 @@ public PubSubMessage<KafkaKey, KafkaMessageEnvelope, Long> deserialize(
valueSerializer.deserialize(valueBytes, providedProtocolSchema, getEnvelope(key.getKeyHeaderByte()));
} catch (Exception e) {
lluwm marked this conversation as resolved.
Show resolved Hide resolved
// Improper header... will ignore.
LOGGER.warn("Received unparsable schema in protocol header: " + VENICE_TRANSPORT_PROTOCOL_HEADER, e);
LOGGER.warn(
"Received unparsable schema or encountered schema registration issue in protocol header: "
+ VENICE_TRANSPORT_PROTOCOL_HEADER,
e);
}
break; // We don't look at other headers
}
Expand Down Expand Up @@ -97,4 +100,9 @@ private KafkaMessageEnvelope getEnvelope(byte keyHeaderByte) {
throw new IllegalStateException("Illegal key header byte: " + keyHeaderByte);
}
}

// For testing only.
public KafkaValueSerializer getValueSerializer() {
return valueSerializer;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -7,14 +7,14 @@
import com.linkedin.venice.schema.SchemaReader;
import com.linkedin.venice.serialization.VeniceKafkaSerializer;
import com.linkedin.venice.utils.ByteUtils;
import com.linkedin.venice.utils.SparseConcurrentList;
import com.linkedin.venice.utils.SparseConcurrentListWithOffset;
import com.linkedin.venice.utils.Utils;
import it.unimi.dsi.fastutil.ints.IntLinkedOpenHashSet;
import it.unimi.dsi.fastutil.ints.IntSet;
import java.io.ByteArrayOutputStream;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.util.List;
import java.util.Map;
import java.util.Properties;
import java.util.function.BiConsumer;
Expand Down Expand Up @@ -93,7 +93,7 @@ private static class ReusableObjects {
private final SpecificDatumWriter writer;

/** Maintains the mapping between protocol version and the corresponding {@link SpecificDatumReader<SPECIFIC_RECORD>} */
private final List<VeniceSpecificDatumReader<SPECIFIC_RECORD>> protocolVersionToReader;
private final SparseConcurrentList<VeniceSpecificDatumReader<SPECIFIC_RECORD>> protocolVersionToReader;

/** The schema of the {@link SpecificRecord} which is compiled in the current version of the code. */
private final Schema compiledProtocol;
Expand Down Expand Up @@ -355,11 +355,28 @@ public SPECIFIC_RECORD deserialize(byte[] bytes, int protocolVersion, SPECIFIC_R

public SPECIFIC_RECORD deserialize(byte[] bytes, Schema providedProtocolSchema, SPECIFIC_RECORD reuse) {
int protocolVersion = getProtocolVersion(bytes);
VeniceSpecificDatumReader<SPECIFIC_RECORD> specificDatumReader = protocolVersionToReader.get(protocolVersion);
if (specificDatumReader == null) {
specificDatumReader = cacheDatumReader(protocolVersion, providedProtocolSchema);
newSchemaEncountered.accept(protocolVersion, providedProtocolSchema);
}

/**
* When a new schema version is discovered during data ingestion, there will be only one thread registering
* the new schema for all the users of the shared InternalAvroSpecificSerializer object. During this period
* of registration, other concurrent threads that calling deserialize will be blocked until the registration
* of the first thread is finished. (It would help to avoid the concurrent request loads in the case e.g. when
* all shared consumer threads in one venice server discover an unknown schema from header, and all the
* threads decide to register this schema to controller.)
*
* It is important to notice that once the registration part is completed, it still allows the deserialization
* to run in parallel.
*
* Also notice that if exception happens in newSchemaEncountered, local protocolVersionToReader will not
* be updated, and it gives its caller an opportunity to discover the failure and perform any
* remediation steps (e.g. retry etc.).
*/
VeniceSpecificDatumReader<SPECIFIC_RECORD> specificDatumReader =
protocolVersionToReader.computeIfAbsent(protocolVersion, index -> {
newSchemaEncountered.accept(protocolVersion, providedProtocolSchema);
return cacheDatumReader(protocolVersion, providedProtocolSchema);
});

return deserialize(bytes, specificDatumReader, reuse);
}

Expand Down Expand Up @@ -426,4 +443,14 @@ private VeniceSpecificDatumReader<SPECIFIC_RECORD> cacheDatumReader(int protocol
private String getCurrentlyLoadedProtocolVersions() {
return knownProtocols().toString();
}

// For testing only.
public void removeAllSchemas() {
this.protocolVersionToReader.clear();
}

// For testing purpose only.
public int getProtocolVersionSize() {
return protocolVersionToReader.size();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -15,6 +15,7 @@
import com.linkedin.venice.controllerapi.UpdateStoreQueryParams;
import com.linkedin.venice.exceptions.ErrorType;
import com.linkedin.venice.exceptions.VeniceException;
import com.linkedin.venice.schema.avro.DirectionalSchemaCompatibilityType;
import com.linkedin.venice.schema.writecompute.WriteComputeSchemaConverter;
import com.linkedin.venice.security.SSLFactory;
import com.linkedin.venice.serialization.avro.AvroProtocolDefinition;
Expand Down Expand Up @@ -76,7 +77,7 @@ public ControllerClientBackedSystemSchemaInitializer(
this.enforceSslOnly = enforceSslOnly;
}

public void execute() {
public void execute(Map<Integer, Schema> inputSchemas) {
if (controllerClient == null) {
if (!controllerUrl.isEmpty()) {
controllerClient = ControllerClient.constructClusterControllerClient(clusterName, controllerUrl, sslFactory);
Expand All @@ -100,14 +101,16 @@ public void execute() {
}

String storeName = protocolDefinition.getSystemStoreName();
Map<Integer, Schema> schemasInLocalResources = Utils.getAllSchemasFromResources(protocolDefinition);
boolean isSchemaResourceInLocal = inputSchemas == null;
Map<Integer, Schema> schemaResources =
isSchemaResourceInLocal ? Utils.getAllSchemasFromResources(protocolDefinition) : inputSchemas;
D2ServiceDiscoveryResponse discoveryResponse = controllerClient.retryableRequest(
DEFAULT_RETRY_TIMES,
c -> c.discoverCluster(storeName),
r -> ErrorType.STORE_NOT_FOUND.equals(r.getErrorType()));
if (discoveryResponse.isError()) {
if (ErrorType.STORE_NOT_FOUND.equals(discoveryResponse.getErrorType())) {
checkAndMayCreateSystemStore(storeName, schemasInLocalResources.get(1));
checkAndMayCreateSystemStore(storeName, schemaResources.get(1));
} else {
throw new VeniceException(
"Error when discovering system store " + storeName + " after retries. Error: "
Expand Down Expand Up @@ -139,22 +142,65 @@ public void execute() {
Map<Integer, Schema> schemasInZk = new HashMap<>();
Arrays.stream(multiSchemaResponse.getSchemas())
.forEach(schema -> schemasInZk.put(schema.getId(), AvroCompatibilityHelper.parse(schema.getSchemaStr())));

if (isSchemaResourceInLocal) {
registerLocalSchemaResources(storeName, schemaResources, schemasInZk);
} else {
// For passed in new schemas, its version could be larger than protocolDefinition.getCurrentProtocolVersion(),
// register schema directly.
for (Map.Entry<Integer, Schema> entry: schemaResources.entrySet()) {
checkAndMayRegisterValueSchema(
storeName,
entry.getKey(),
schemasInZk.get(entry.getKey()),
entry.getValue(),
determineSchemaCompatabilityType());
}
}
}

private void registerLocalSchemaResources(
String storeName,
Map<Integer, Schema> schemaResources,
Map<Integer, Schema> schemasInZk) {
for (int version = 1; version <= protocolDefinition.getCurrentProtocolVersion(); version++) {
Schema schemaInLocalResources = schemasInLocalResources.get(version);
Schema schemaInLocalResources = schemaResources.get(version);
if (schemaInLocalResources == null) {
throw new VeniceException(
"Invalid protocol definition: " + protocolDefinition.name() + " does not have a version " + version
+ " even though it is less than or equal to the current version ("
+ protocolDefinition.getCurrentProtocolVersion() + ").");
}
checkAndMayRegisterValueSchema(storeName, version, schemasInZk.get(version), schemaInLocalResources);
} else {
checkAndMayRegisterValueSchema(
storeName,
version,
schemasInZk.get(version),
schemaInLocalResources,
determineSchemaCompatabilityType());

if (autoRegisterPartialUpdateSchema) {
checkAndMayRegisterPartialUpdateSchema(storeName, version, schemaInLocalResources);
if (autoRegisterPartialUpdateSchema) {
checkAndMayRegisterPartialUpdateSchema(storeName, version, schemaInLocalResources);
}
}
}
}

DirectionalSchemaCompatibilityType determineSchemaCompatabilityType() {
if (protocolDefinition == AvroProtocolDefinition.METADATA_SYSTEM_SCHEMA_STORE) {
return DirectionalSchemaCompatibilityType.FULL;
}

if (protocolDefinition == AvroProtocolDefinition.KAFKA_MESSAGE_ENVELOPE) {
return DirectionalSchemaCompatibilityType.BACKWARD;
}

return DirectionalSchemaCompatibilityType.FULL;
}

public void execute() {
execute(null);
}

private boolean hasLeaderController() {
try {
RetryUtils.executeWithMaxAttempt(
Expand Down Expand Up @@ -236,11 +282,13 @@ private void checkAndMayRegisterValueSchema(
String storeName,
int valueSchemaId,
Schema schemaInZk,
Schema schemaInLocalResources) {
Schema schemaInLocalResources,
DirectionalSchemaCompatibilityType compatType) {
if (schemaInZk == null) {
SchemaResponse addValueSchemaResponse = controllerClient.retryableRequest(
DEFAULT_RETRY_TIMES,
c -> c.addValueSchema(storeName, schemaInLocalResources.toString(), valueSchemaId));
c -> c.addValueSchema(storeName, schemaInLocalResources.toString(), valueSchemaId, compatType));

if (addValueSchemaResponse.isError()) {
throw new VeniceException(
"Error when adding value schema " + valueSchemaId + " to system store " + storeName + " in cluster "
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,16 +16,19 @@
import com.linkedin.venice.controllerapi.StoreResponse;
import com.linkedin.venice.exceptions.ErrorType;
import com.linkedin.venice.exceptions.VeniceException;
import com.linkedin.venice.schema.avro.DirectionalSchemaCompatibilityType;
import com.linkedin.venice.serialization.avro.AvroProtocolDefinition;
import java.util.Optional;
import org.testng.Assert;
import org.testng.annotations.Test;


public class ControllerClientBackedSystemSchemaInitializerTest {
ControllerClientBackedSystemSchemaInitializer initializer;

@Test
public void testCreateSystemStoreAndRegisterSchema() {
ControllerClientBackedSystemSchemaInitializer initializer;

try {
initializer = new ControllerClientBackedSystemSchemaInitializer(
AvroProtocolDefinition.METADATA_SYSTEM_SCHEMA_STORE,
Expand Down Expand Up @@ -70,13 +73,36 @@ public void testCreateSystemStoreAndRegisterSchema() {
doReturn(new MultiSchemaResponse.Schema[0]).when(multiSchemaResponse).getSchemas();
doReturn(multiSchemaResponse).when(controllerClient).getAllValueSchema(any());
SchemaResponse schemaResponse = mock(SchemaResponse.class);
doReturn(schemaResponse).when(controllerClient).addValueSchema(any(), any(), anyInt());
doReturn(schemaResponse).when(controllerClient).addValueSchema(any(), any(), anyInt(), any());
doCallRealMethod().when(controllerClient).retryableRequest(anyInt(), any(), any());
doCallRealMethod().when(controllerClient).retryableRequest(anyInt(), any());
initializer.setControllerClient(controllerClient);
initializer.execute();
verify(controllerClient, times(1)).createNewSystemStore(any(), any(), any(), any());
verify(controllerClient, times(AvroProtocolDefinition.METADATA_SYSTEM_SCHEMA_STORE.getCurrentProtocolVersion()))
.addValueSchema(any(), any(), anyInt());
.addValueSchema(any(), any(), anyInt(), any());
}

@Test
public void testSchemaCompatabilityType() {
for (AvroProtocolDefinition protocol: AvroProtocolDefinition.values()) {
initializer = new ControllerClientBackedSystemSchemaInitializer(
protocol,
"testCluster",
null,
null,
false,
Optional.empty(),
"",
"",
"",
false);
if (protocol == AvroProtocolDefinition.KAFKA_MESSAGE_ENVELOPE) {
Assert
.assertEquals(initializer.determineSchemaCompatabilityType(), DirectionalSchemaCompatibilityType.BACKWARD);
} else {
Assert.assertEquals(initializer.determineSchemaCompatabilityType(), DirectionalSchemaCompatibilityType.FULL);
}
}
}
}
Loading