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

Independent schema is set for each consumer generated by topic #6356

Merged
merged 21 commits into from
Mar 6, 2020
Merged
Show file tree
Hide file tree
Changes from 16 commits
Commits
Show all changes
21 commits
Select commit Hold shift + click to select a range
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
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;
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

missing license header


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 All @@ -29,36 +29,36 @@ public class Schemas {
@NoArgsConstructor
@AllArgsConstructor
public static class PersonOne{
int id;
public int id;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Could you please explain why the access modifiers were switched to public?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

SchemaCompatibilityCheckTest also used it.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This class provides all possible constructors and getters/setters for all fields. Please use them instead of defining fields as public

}

@Data
@AllArgsConstructor
@NoArgsConstructor
public static class PersonTwo{
int id;
public int id;

@AvroDefault("\"Tom\"")
String name;
public String name;
}

@Data
@AllArgsConstructor
@NoArgsConstructor
public static class PersonThree{
int id;
public int id;

String name;
public String name;
}

@Data
@AllArgsConstructor
@NoArgsConstructor
public static class PersonFour{
int id;
public int id;

String name;
public String name;

int age;
public int age;
}
}
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
Original file line number Diff line number Diff line change
Expand Up @@ -136,6 +136,15 @@ default void configureSchemaInfo(String topic, String componentName,
// no-op
}

/**
* Clone schema.
*
* @return cloned schema.
*/
default Schema<T> cloneSchema() {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Did you consider to use the standard Cloneable interface with the clone method?

Copy link
Contributor Author

@congbobo184 congbobo184 Feb 28, 2020

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It seems don't need to add the flag to control it.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@congbobo184 do you mean that Cloneable interface doesn't return Schema<T>?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

My understanding of what @vzhikserg said deviated, now I understand what he said. I will extends the Cloneable.

return this;
}

/**
* 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 @@ -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 @@ -331,7 +331,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 @@ -444,7 +444,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 @@ -764,8 +764,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 @@ -775,11 +775,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.cloneSchema();
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 @@ -788,18 +789,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 @@ -132,6 +132,11 @@ public void configureSchemaInfo(String topicName,
}
}

@Override
public Schema<GenericRecord> cloneSchema() {
return Schema.AUTO_CONSUME();
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

When you implementing cloneSchema, don't you need to copy the fields to the new instance?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

�it don't need to copy fields, AutoConsumeSchema represent it can auto consume, if you want clone the fields you can clone the true schema in AutoConsumeSchema. After all, only its schema is changeable.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I understand it can be changed. but cloneSchema is cloning a schema, we should copy the fields when cloning a schema.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

you are right, we should clone the fields when clone a schema

}

private GenericSchema generateSchema(SchemaInfo schemaInfo) {
if (schemaInfo.getType() != SchemaType.AVRO
&& schemaInfo.getType() != SchemaType.JSON) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@
import org.apache.avro.Conversions;
import org.apache.avro.data.TimeConversions;
import org.apache.avro.reflect.ReflectData;
import org.apache.pulsar.client.api.Schema;
import org.apache.pulsar.client.api.schema.SchemaDefinition;
import org.apache.pulsar.client.api.schema.SchemaReader;
import org.apache.pulsar.client.impl.schema.reader.AvroReader;
Expand Down Expand Up @@ -75,6 +76,11 @@ public boolean supportSchemaVersioning() {
return true;
}

@Override
public Schema<T> cloneSchema() {
return new AvroSchema<>(schemaInfo);
}

public static <T> AvroSchema<T> of(SchemaDefinition<T> schemaDefinition) {
return new AvroSchema<>(parseSchemaInfo(schemaDefinition, SchemaType.AVRO));
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -193,6 +193,11 @@ public void configureSchemaInfo(String topicName,
}
}

@Override
public Schema<KeyValue<K, V>> cloneSchema() {
return KeyValueSchema.of(keySchema.cloneSchema(), valueSchema.cloneSchema(), keyValueEncodingType);
}

private void configureKeyValueSchemaInfo() {
this.schemaInfo = KeyValueSchemaInfo.encodeKeyValueSchemaInfo(
keySchema, valueSchema, keyValueEncodingType
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -54,6 +54,11 @@ public boolean supportSchemaVersioning() {
return true;
}

@Override
public org.apache.pulsar.client.api.Schema<GenericRecord> cloneSchema() {
return GenericAvroSchema.of(schemaInfo, useProvidedSchemaAsReaderSchema);
}

@Override
protected SchemaReader<GenericRecord> loadReader(BytesSchemaVersion schemaVersion) {
SchemaInfo schemaInfo = getSchemaInfoByVersion(schemaVersion.get());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -76,5 +76,4 @@ public static GenericSchemaImpl of(SchemaInfo schemaInfo,
+ schemaInfo.getType() + "'");
}
}

}