On the client side, we add AutoConsume enum in the Schema.Type, and add protocol version in the ProtocolVersion.
On the broker side, the SchemaType containing AUTO_CONSUME is recorded in org.apache.pulsar.broker.service.Consumer.
API Changes
Protocal change: Schema.Type and ProtocolVersion
message Schema {
enum Type {
AutoConsume = 21;
}
}
enum ProtocolVersion {
v21 = 21; // Carry the AUTO_CONSUME schema to the Broker after this version
}
Record SchemaType in Consumer
@Getter
private final SchemaType schemaType;
Implementation
On the client side
Set "default" schemaInfo(schemaType=SchemaType.AUTO_CONSUME) in org.apache.pulsar.client.impl.schema.AutoConsumeSchema.
Get and set SchemaInfo with AUTO_CONSUME schema in org.apache.pulsar.common.protocol.Commands#newSubscribe when schemaType is AUTO_CONSUME and proto version is greater than or equal to v21.
On the broker side
When the schema is not null and schemaType not AUTO_CONSUME, then addSchemaIfIdleOrCheckCompatible.
Get the schemaType and record it in Consumer.
The active consumers schema of the Topic have one or more consumers schema whose not AUTO_CONSUME. Then checkSchemaCompatibleForConsumer.
Motivation
Fixed the failure to use schema to create consumer after using AUTO-CONSUME consumer to subscribe an empty topic, and Broker returned the error message as IncompatibleSchemaException("Topic does not have schema to check"). https://github.com/apache/pulsar/blob/ed33fb399e661e4d47baeaaa8d0cdb3bfadc9546/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java#L1037 https://github.com/apache/pulsar/blob/ed33fb399e661e4d47baeaaa8d0cdb3bfadc9546/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java#L1147-L1152 https://github.com/apache/pulsar/blob/ed33fb399e661e4d47baeaaa8d0cdb3bfadc9546/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java#L3054-L3071 https://github.com/apache/pulsar/blob/ed33fb399e661e4d47baeaaa8d0cdb3bfadc9546/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentTopic.java#L1162-L1177 We should record whether the active consumers of the Topic have one or more consumers whose SchemaType is not AUTO_CONSUME.
Goal
org.apache.pulsar.broker.service.Consumer
.API Changes
Protocal change: Schema.Type and ProtocolVersion
Record SchemaType in Consumer
Implementation
On the client side
org.apache.pulsar.client.impl.schema.AutoConsumeSchema
.org.apache.pulsar.common.protocol.Commands#newSubscribe
when schemaType is AUTO_CONSUME and proto version is greater than or equal tov21
.On the broker side
checkSchemaCompatibleForConsumer
.Alternatives
On the client side, we add an optional field in the CommandSubscribe.
add optional bool check_schema_compatibility = 20 [default = true];
On the broker side, record checkSchemaCompatibility in
org.apache.pulsar.broker.service.Consumer
.If checkSchemaCompatibility is true, schema compatibility check is required. If schemaType is AUTO_CONSUME, the checkSchemaCompatibility is false.
Anything else?
No response
Links
Discussion: https://lists.apache.org/thread/v7p88h7grqnbzocw34g6jvxjfw962kfd Vote: https://lists.apache.org/thread/pvcdlbflofoj41ryo1lrn0zlhj15bwpv PR: #17449