Skip to content

Commit

Permalink
Independent schema is set for each consumer generated by topic (apach…
Browse files Browse the repository at this point in the history
…e#6356)

### Motivation

Master Issue: apache#5454 

When one Consumer subscribe multi topic, setSchemaInfoPorvider() will be covered by the consumer generated by the last topic.

### Modification
clone schema for each consumer generated by topic.
### Verifying this change
Add the schemaTest for it.
  • Loading branch information
congbobo184 authored Mar 6, 2020
1 parent f33567e commit 8003d08
Show file tree
Hide file tree
Showing 18 changed files with 244 additions and 27 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -124,5 +124,10 @@ public static <T> OldJSONSchema<T> of(Class<T> pojo, Map<String, String> propert
info.setSchema(mapper.writeValueAsBytes(schema));
return new OldJSONSchema<>(info, pojo, mapper);
}

@Override
public Schema<T> clone() {
return this;
}
}
}
136 changes: 136 additions & 0 deletions pulsar-broker/src/test/java/org/apache/pulsar/schema/SchemaTest.java
Original file line number Diff line number Diff line change
@@ -0,0 +1,136 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you 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 org.apache.pulsar.schema;

import com.google.common.collect.Sets;
import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest;
import org.apache.pulsar.client.api.Consumer;
import org.apache.pulsar.client.api.Producer;
import org.apache.pulsar.client.api.Schema;
import org.apache.pulsar.client.api.schema.SchemaDefinition;
import org.apache.pulsar.common.naming.TopicDomain;
import org.apache.pulsar.common.naming.TopicName;
import org.apache.pulsar.common.policies.data.ClusterData;
import org.apache.pulsar.common.policies.data.TenantInfo;
import org.testng.annotations.AfterMethod;
import org.testng.annotations.BeforeMethod;
import org.testng.annotations.Test;

import java.util.Collections;

import static org.apache.pulsar.common.naming.TopicName.PUBLIC_TENANT;
import static org.apache.pulsar.schema.compatibility.SchemaCompatibilityCheckTest.randomName;
import static org.junit.Assert.assertEquals;

public class SchemaTest extends MockedPulsarServiceBaseTest {

private final static String CLUSTER_NAME = "test";

@BeforeMethod
@Override
public void setup() throws Exception {
super.internalSetup();

// Setup namespaces
admin.clusters().createCluster(CLUSTER_NAME, new ClusterData(pulsar.getBrokerServiceUrl()));
TenantInfo tenantInfo = new TenantInfo();
tenantInfo.setAllowedClusters(Collections.singleton(CLUSTER_NAME));
admin.tenants().createTenant(PUBLIC_TENANT, tenantInfo);
}

@AfterMethod
@Override
public void cleanup() throws Exception {
super.internalCleanup();
}

@Test
public void testMultiTopicSetSchemaProvider() throws Exception {
final String tenant = PUBLIC_TENANT;
final String namespace = "test-namespace-" + randomName(16);
final String topicOne = "test-multi-version-schema-one";
final String topicTwo = "test-multi-version-schema-two";
final String fqtnOne = TopicName.get(
TopicDomain.persistent.value(),
tenant,
namespace,
topicOne
).toString();

final String fqtnTwo = TopicName.get(
TopicDomain.persistent.value(),
tenant,
namespace,
topicTwo
).toString();


admin.namespaces().createNamespace(
tenant + "/" + namespace,
Sets.newHashSet(CLUSTER_NAME)
);

admin.topics().createPartitionedTopic(fqtnOne, 3);
admin.topics().createPartitionedTopic(fqtnTwo, 3);

admin.schemas().createSchema(fqtnOne, Schema.AVRO(
SchemaDefinition.<Schemas.PersonOne>builder().withAlwaysAllowNull
(false).withSupportSchemaVersioning(true).
withPojo(Schemas.PersonOne.class).build()).getSchemaInfo());

admin.schemas().createSchema(fqtnOne, Schema.AVRO(
SchemaDefinition.<Schemas.PersonTwo>builder().withAlwaysAllowNull
(false).withSupportSchemaVersioning(true).
withPojo(Schemas.PersonTwo.class).build()).getSchemaInfo());

admin.schemas().createSchema(fqtnTwo, Schema.AVRO(
SchemaDefinition.<Schemas.PersonTwo>builder().withAlwaysAllowNull
(false).withSupportSchemaVersioning(true).
withPojo(Schemas.PersonTwo.class).build()).getSchemaInfo());

Producer<Schemas.PersonTwo> producer = pulsarClient.newProducer(Schema.AVRO(
SchemaDefinition.<Schemas.PersonTwo>builder().withAlwaysAllowNull
(false).withSupportSchemaVersioning(true).
withPojo(Schemas.PersonTwo.class).build()))
.topic(fqtnOne)
.create();

Schemas.PersonTwo personTwo = new Schemas.PersonTwo();
personTwo.setId(1);
personTwo.setName("Tom");


Consumer<Schemas.PersonTwo> consumer = pulsarClient.newConsumer(Schema.AVRO(
SchemaDefinition.<Schemas.PersonTwo>builder().withAlwaysAllowNull
(false).withSupportSchemaVersioning(true).
withPojo(Schemas.PersonTwo.class).build()))
.subscriptionName("test")
.topic(fqtnOne, fqtnTwo)
.subscribe();

producer.send(personTwo);

Schemas.PersonTwo personConsume = consumer.receive().getValue();
assertEquals("Tom", personConsume.getName());
assertEquals(1, personConsume.getId());

producer.close();
consumer.close();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -16,7 +16,7 @@
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.pulsar.schema.compatibility;
package org.apache.pulsar.schema;

import lombok.AllArgsConstructor;
import lombok.Data;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -35,6 +35,7 @@
import org.apache.pulsar.common.policies.data.ClusterData;
import org.apache.pulsar.common.policies.data.SchemaCompatibilityStrategy;
import org.apache.pulsar.common.policies.data.TenantInfo;
import org.apache.pulsar.schema.Schemas;
import org.testng.Assert;
import org.testng.annotations.AfterMethod;
import org.testng.annotations.BeforeMethod;
Expand All @@ -46,7 +47,6 @@

import static org.apache.pulsar.common.naming.TopicName.PUBLIC_TENANT;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;


@Slf4j
Expand Down Expand Up @@ -141,7 +141,7 @@ public void testConsumerCompatibilityCheckCanReadLastTest(SchemaCompatibilityStr


Schemas.PersonOne personOne = new Schemas.PersonOne();
personOne.id = 1;
personOne.setId(1);

producerOne.send(personOne);
Message<Schemas.PersonThree> message = null;
Expand All @@ -162,16 +162,16 @@ public void testConsumerCompatibilityCheckCanReadLastTest(SchemaCompatibilityStr
.create();

Schemas.PersonTwo personTwo = new Schemas.PersonTwo();
personTwo.id = 1;
personTwo.name = "Jerry";
personTwo.setId(1);
personTwo.setName("Jerry");
producerTwo.send(personTwo);

message = consumerThree.receive();
Schemas.PersonThree personThree = message.getValue();
consumerThree.acknowledge(message);

assertEquals(personThree.id, 1);
assertEquals(personThree.name, "Jerry");
assertEquals(personThree.getId(), 1);
assertEquals(personThree.getName(), "Jerry");

consumerThree.close();
producerOne.close();
Expand Down Expand Up @@ -270,8 +270,8 @@ public void testIsAutoUpdateSchema(SchemaCompatibilityStrategy schemaCompatibili
Schemas.PersonTwo personTwo = message.getValue();
consumerTwo.acknowledge(message);

assertEquals(personTwo.id, 2);
assertEquals(personTwo.name, "Lucy");
assertEquals(personTwo.getId(), 2);
assertEquals(personTwo.getName(), "Lucy");

producer.close();
consumerTwo.close();
Expand All @@ -287,8 +287,8 @@ public void testIsAutoUpdateSchema(SchemaCompatibilityStrategy schemaCompatibili
personTwo = message.getValue();
consumerTwo.acknowledge(message);

assertEquals(personTwo.id, 2);
assertEquals(personTwo.name, "Lucy");
assertEquals(personTwo.getId(), 2);
assertEquals(personTwo.getName(), "Lucy");

consumerTwo.close();
producer.close();
Expand Down Expand Up @@ -338,7 +338,7 @@ public void testProducerSendWithOldSchemaAndConsumerCanRead(SchemaCompatibilityS
Message<Schemas.PersonOne> message = consumerOne.receive();
personOne = message.getValue();

assertEquals(10, personOne.id);
assertEquals(10, personOne.getId());

consumerOne.close();
producerOne.close();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -35,7 +35,7 @@
/**
* Message schema definition.
*/
public interface Schema<T> {
public interface Schema<T> extends Cloneable{

/**
* Check if the message is a valid object for this schema.
Expand Down Expand Up @@ -136,6 +136,13 @@ default void configureSchemaInfo(String topic, String componentName,
// no-op
}

/**
* Duplicates the schema.
*
* @return The duplicated schema.
*/
Schema<T> clone();

/**
* Schema that doesn't perform any encoding on the message payloads. Accepts a byte array and it passes it through.
*/
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -74,4 +74,9 @@ public T decode(byte[] message) {
public SchemaInfo getSchemaInfo() {
return Schema.BYTES.getSchemaInfo();
}

@Override
public Schema<T> clone() {
return this;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -74,4 +74,9 @@ public T decode(byte[] message) {
public SchemaInfo getSchemaInfo() {
return Schema.BYTES.getSchemaInfo();
}

@Override
public Schema<T> clone() {
return this;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -800,16 +800,17 @@ private CompletableFuture<Void> subscribeAsync(String topicName, int numberParti

private void subscribeTopicPartitions(CompletableFuture<Void> subscribeResult, String topicName, int numPartitions,
boolean createIfDoesNotExist) {
client.preProcessSchemaBeforeSubscribe(client, schema, topicName).whenComplete((ignored, cause) -> {
client.preProcessSchemaBeforeSubscribe(client, schema, topicName).whenComplete((schema, cause) -> {
if (null == cause) {
doSubscribeTopicPartitions(subscribeResult, topicName, numPartitions, createIfDoesNotExist);
doSubscribeTopicPartitions(schema, subscribeResult, topicName, numPartitions, createIfDoesNotExist);
} else {
subscribeResult.completeExceptionally(cause);
}
});
}

private void doSubscribeTopicPartitions(CompletableFuture<Void> subscribeResult, String topicName, int numPartitions,
private void doSubscribeTopicPartitions(Schema<T> schema,
CompletableFuture<Void> subscribeResult, String topicName, int numPartitions,
boolean createIfDoesNotExist) {
if (log.isDebugEnabled()) {
log.debug("Subscribe to topic {} metadata.partitions: {}", topicName, numPartitions);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -335,7 +335,7 @@ public <T> CompletableFuture<Consumer<T>> subscribeAsync(ConsumerConfigurationDa

private <T> CompletableFuture<Consumer<T>> singleTopicSubscribeAsync(ConsumerConfigurationData<T> conf, Schema<T> schema, ConsumerInterceptors<T> interceptors) {
return preProcessSchemaBeforeSubscribe(this, schema, conf.getSingleTopic())
.thenCompose(ignored -> doSingleTopicSubscribeAsync(conf, schema, interceptors));
.thenCompose(schemaClone -> doSingleTopicSubscribeAsync(conf, schemaClone, interceptors));
}

private <T> CompletableFuture<Consumer<T>> doSingleTopicSubscribeAsync(ConsumerConfigurationData<T> conf, Schema<T> schema, ConsumerInterceptors<T> interceptors) {
Expand Down Expand Up @@ -448,7 +448,7 @@ public CompletableFuture<Reader<byte[]>> createReaderAsync(ReaderConfigurationDa

public <T> CompletableFuture<Reader<T>> createReaderAsync(ReaderConfigurationData<T> conf, Schema<T> schema) {
return preProcessSchemaBeforeSubscribe(this, schema, conf.getTopicName())
.thenCompose(ignored -> doCreateReaderAsync(conf, schema));
.thenCompose(schemaClone -> doCreateReaderAsync(conf, schemaClone));
}

<T> CompletableFuture<Reader<T>> doCreateReaderAsync(ReaderConfigurationData<T> conf, Schema<T> schema) {
Expand Down Expand Up @@ -768,8 +768,8 @@ private LoadingCache<String, SchemaInfoProvider> getSchemaProviderLoadingCache()
}

@SuppressWarnings("unchecked")
protected CompletableFuture<Void> preProcessSchemaBeforeSubscribe(PulsarClientImpl pulsarClientImpl,
Schema schema,
protected <T> CompletableFuture<Schema<T>> preProcessSchemaBeforeSubscribe(PulsarClientImpl pulsarClientImpl,
Schema<T> schema,
String topicName) {
if (schema != null && schema.supportSchemaVersioning()) {
final SchemaInfoProvider schemaInfoProvider;
Expand All @@ -779,11 +779,12 @@ protected CompletableFuture<Void> preProcessSchemaBeforeSubscribe(PulsarClientIm
log.error("Failed to load schema info provider for topic {}", topicName, e);
return FutureUtil.failedFuture(e.getCause());
}

schema = schema.clone();
if (schema.requireFetchingSchemaInfo()) {
Schema finalSchema = schema;
return schemaInfoProvider.getLatestSchema().thenCompose(schemaInfo -> {
if (null == schemaInfo) {
if (!(schema instanceof AutoConsumeSchema)) {
if (!(finalSchema instanceof AutoConsumeSchema)) {
// no schema info is found
return FutureUtil.failedFuture(
new PulsarClientException.NotFoundException(
Expand All @@ -792,18 +793,18 @@ protected CompletableFuture<Void> preProcessSchemaBeforeSubscribe(PulsarClientIm
}
try {
log.info("Configuring schema for topic {} : {}", topicName, schemaInfo);
schema.configureSchemaInfo(topicName, "topic", schemaInfo);
finalSchema.configureSchemaInfo(topicName, "topic", schemaInfo);
} catch (RuntimeException re) {
return FutureUtil.failedFuture(re);
}
schema.setSchemaInfoProvider(schemaInfoProvider);
return CompletableFuture.completedFuture(null);
finalSchema.setSchemaInfoProvider(schemaInfoProvider);
return CompletableFuture.completedFuture(finalSchema);
});
} else {
schema.setSchemaInfoProvider(schemaInfoProvider);
}
}
return CompletableFuture.completedFuture(null);
return CompletableFuture.completedFuture(schema);
}

//
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -61,4 +61,9 @@ T decode(ByteBuf byteBuf, byte[] schemaVersion) {
// ignore version by default (most of the primitive schema implementations ignore schema version)
return decode(byteBuf);
}

@Override
public Schema<T> clone() {
return this;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -132,6 +132,20 @@ public void configureSchemaInfo(String topicName,
}
}

@Override
public Schema<GenericRecord> clone() {
Schema<GenericRecord> schema = Schema.AUTO_CONSUME();
if (this.schema != null) {
schema.configureSchemaInfo(topicName, componentName, this.schema.getSchemaInfo());
} else {
schema.configureSchemaInfo(topicName, componentName, null);
}
if (schemaInfoProvider != null) {
schema.setSchemaInfoProvider(schemaInfoProvider);
}
return schema;
}

private GenericSchema generateSchema(SchemaInfo schemaInfo) {
if (schemaInfo.getType() != SchemaType.AVRO
&& schemaInfo.getType() != SchemaType.JSON) {
Expand Down
Loading

0 comments on commit 8003d08

Please sign in to comment.