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

[schemaregistry]ProtobufNative Schema Support #8372

Merged
merged 12 commits into from
Nov 17, 2020
Original file line number Diff line number Diff line change
Expand Up @@ -239,7 +239,7 @@ public class ServiceConfiguration implements PulsarConfiguration {
@FieldContext(
category = CATEGORY_SERVER,
dynamic = true,
doc = "The maximum number of tenants that each pulsar cluster can create."
doc = "The maximum number of tenants that each pulsar cluster can create."
+ "This configuration is not precise control, in a concurrent scenario, the threshold will be exceeded."
)
private int maxTenants = 0;
Expand Down Expand Up @@ -430,9 +430,9 @@ public class ServiceConfiguration implements PulsarConfiguration {
doc = "When a namespace is created without specifying the number of bundle, this"
+ " value will be used as the default")
private int defaultNumberOfNamespaceBundles = 4;

@FieldContext(
category = CATEGORY_POLICIES,
category = CATEGORY_POLICIES,
dynamic = true,
doc = "The maximum number of namespaces that each tenant can create."
+ "This configuration is not precise control, in a concurrent scenario, the threshold will be exceeded")
Expand Down Expand Up @@ -1689,7 +1689,8 @@ public class ServiceConfiguration implements PulsarConfiguration {
)
private Set<String> schemaRegistryCompatibilityCheckers = Sets.newHashSet(
"org.apache.pulsar.broker.service.schema.JsonSchemaCompatibilityCheck",
"org.apache.pulsar.broker.service.schema.AvroSchemaCompatibilityCheck"
"org.apache.pulsar.broker.service.schema.AvroSchemaCompatibilityCheck",
"org.apache.pulsar.broker.service.schema.ProtobufNativeSchemaCompatibilityCheck"
);

/**** --- WebSocket --- ****/
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,72 @@
/**
* 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.broker.service.schema;

import org.apache.pulsar.broker.service.schema.exceptions.IncompatibleSchemaException;
import org.apache.pulsar.client.impl.schema.ProtobufNativeSchemaUtils;
import org.apache.pulsar.common.policies.data.SchemaCompatibilityStrategy;
import org.apache.pulsar.common.protocol.schema.SchemaData;
import org.apache.pulsar.common.schema.SchemaType;

import static com.google.protobuf.Descriptors.Descriptor;

/**
* The {@link SchemaCompatibilityCheck} implementation for {@link SchemaType#PROTOBUF_NATIVE}.
*/
public class ProtobufNativeSchemaCompatibilityCheck implements SchemaCompatibilityCheck {

@Override
public SchemaType getSchemaType() {
return SchemaType.PROTOBUF_NATIVE;
}

@Override
public void checkCompatible(SchemaData from, SchemaData to, SchemaCompatibilityStrategy strategy) throws IncompatibleSchemaException {
Descriptor fromDescriptor = ProtobufNativeSchemaUtils.deserialize(from.getData());
Descriptor toDescriptor = ProtobufNativeSchemaUtils.deserialize(to.getData());
switch (strategy) {
case BACKWARD_TRANSITIVE:
case BACKWARD:
case FORWARD_TRANSITIVE:
case FORWARD:
case FULL_TRANSITIVE:
case FULL:
CheckRootRootMessageChange(fromDescriptor, toDescriptor, strategy);
return;
case ALWAYS_COMPATIBLE:
return;
default:
throw new IncompatibleSchemaException("Unknown SchemaCompatibilityStrategy");
}
}

@Override
public void checkCompatible(Iterable<SchemaData> from, SchemaData to, SchemaCompatibilityStrategy strategy) throws IncompatibleSchemaException {
for (SchemaData schemaData : from) {
checkCompatible(schemaData, to, strategy);
}
}

private void CheckRootRootMessageChange(Descriptor fromDescriptor, Descriptor toDescriptor, SchemaCompatibilityStrategy strategy) throws IncompatibleSchemaException {
hnail marked this conversation as resolved.
Show resolved Hide resolved
if (!fromDescriptor.getFullName().equals(toDescriptor.getFullName())) {
throw new IncompatibleSchemaException("Protobuf root message isn't allow change!");
}
}

}
Original file line number Diff line number Diff line change
@@ -0,0 +1,49 @@
/**
* 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.broker.service.schema.validator;

import com.google.protobuf.Descriptors;
import org.apache.pulsar.broker.service.schema.exceptions.InvalidSchemaDataException;
import org.apache.pulsar.client.impl.schema.ProtobufNativeSchemaUtils;
import org.apache.pulsar.common.protocol.schema.SchemaData;

public class ProtobufNativeSchemaDataValidator implements SchemaDataValidator {

@Override
public void validate(SchemaData schemaData) throws InvalidSchemaDataException {
Descriptors.Descriptor descriptor;
try {
descriptor = ProtobufNativeSchemaUtils.deserialize(schemaData.getData());
} catch (Exception e) {
throw new InvalidSchemaDataException("deserialize ProtobufNative Schema failed", e);
hnail marked this conversation as resolved.
Show resolved Hide resolved
}
if (descriptor == null) {
throw new InvalidSchemaDataException("protobuf root message Descriptor is null , please recheck rootMessageTypeName or rootFileDescriptorName conf. ");
hnail marked this conversation as resolved.
Show resolved Hide resolved
}
}

public static ProtobufNativeSchemaDataValidator of() {
return INSTANCE;
}

private static final ProtobufNativeSchemaDataValidator INSTANCE = new ProtobufNativeSchemaDataValidator();

private ProtobufNativeSchemaDataValidator() {
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -41,6 +41,9 @@ static void validateSchemaData(SchemaData schemaData) throws InvalidSchemaDataEx
case PROTOBUF:
StructSchemaDataValidator.of().validate(schemaData);
break;
case PROTOBUF_NATIVE:
ProtobufNativeSchemaDataValidator.of().validate(schemaData);
break;
case STRING:
StringSchemaDataValidator.of().validate(schemaData);
break;
Expand Down
1 change: 1 addition & 0 deletions pulsar-broker/src/main/proto/SchemaRegistryFormat.proto
Original file line number Diff line number Diff line change
Expand Up @@ -44,6 +44,7 @@ message SchemaInfo {
LOCALDATE = 18;
LOCALTIME = 19;
LOCALDATETIME = 20;
PROTOBUFNATIVE = 21;
}
message KeyValuePair {
required string key = 1;
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,50 @@
/**
* 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.broker.service.schema;

import org.apache.pulsar.client.impl.schema.ProtobufNativeSchemaUtils;
import org.apache.pulsar.common.policies.data.SchemaCompatibilityStrategy;
import org.apache.pulsar.common.protocol.schema.SchemaData;
import org.apache.pulsar.common.schema.SchemaType;
import org.testng.Assert;
import org.testng.annotations.Test;

import static com.google.protobuf.Descriptors.Descriptor;

public class ProtobufNativeSchemaCompatibilityCheckTest {

private static final SchemaData schemaData1 = getSchemaData(org.apache.pulsar.client.api.schema.proto.Test.TestMessage.getDescriptor());

private static final SchemaData schemaData2 = getSchemaData(org.apache.pulsar.client.api.schema.proto.Test.SubMessage.getDescriptor());

/**
* make sure protobuf root message isn't allow change
*/
@Test
public void testRootMessageChange() {
ProtobufNativeSchemaCompatibilityCheck compatibilityCheck = new ProtobufNativeSchemaCompatibilityCheck();
Assert.assertFalse(compatibilityCheck.isCompatible(schemaData2, schemaData1,
SchemaCompatibilityStrategy.FULL),
"Protobuf root message isn't allow change");
}

private static SchemaData getSchemaData(Descriptor descriptor) {
return SchemaData.builder().data(ProtobufNativeSchemaUtils.serialize(descriptor)).type(SchemaType.PROTOBUF_NATIVE).build();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -251,6 +251,27 @@ static <T extends com.google.protobuf.GeneratedMessageV3> Schema<T> PROTOBUF(Sch
return DefaultImplementation.newProtobufSchema(schemaDefinition);
}

/**
* Create a Protobuf-Native schema type by extracting the fields of the specified class.
*
* @param clazz the Protobuf generated class to be used to extract the schema
* @return a Schema instance
*/
static <T extends com.google.protobuf.GeneratedMessageV3> Schema<T> PROTOBUFNATIVE(Class<T> clazz) {
hnail marked this conversation as resolved.
Show resolved Hide resolved
return DefaultImplementation.newProtobufNativeSchema(SchemaDefinition.builder().withPojo(clazz).build());
}

/**
* Create a Protobuf-Native schema type with schema definition.
*
* @param schemaDefinition schemaDefinition the definition of the schema
* @return a Schema instance
*/
static <T extends com.google.protobuf.GeneratedMessageV3> Schema<T> PROTOBUFNATIVE(
hnail marked this conversation as resolved.
Show resolved Hide resolved
SchemaDefinition<T> schemaDefinition) {
return DefaultImplementation.newProtobufNativeSchema(schemaDefinition);
}

/**
* Create a Avro schema type by default configuration of the class.
*
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -263,6 +263,14 @@ public static <T extends com.google.protobuf.GeneratedMessageV3> Schema<T> newPr
.invoke(null, schemaDefinition));
}

public static <T extends com.google.protobuf.GeneratedMessageV3> Schema<T> newProtobufNativeSchema(
SchemaDefinition schemaDefinition) {
return catchExceptions(
() -> (Schema<T>) getStaticMethod(
"org.apache.pulsar.client.impl.schema.ProtobufNativeSchema", "of", SchemaDefinition.class)
.invoke(null, schemaDefinition));
}

public static <T> Schema<T> newJSONSchema(SchemaDefinition schemaDefinition) {
return catchExceptions(
() -> (Schema<T>) getStaticMethod(
Expand Down Expand Up @@ -326,10 +334,22 @@ public static Schema<?> getSchema(SchemaInfo schemaInfo) {
}

public static GenericSchema<GenericRecord> getGenericSchema(SchemaInfo schemaInfo) {
return catchExceptions(
() -> (GenericSchema) getStaticMethod(
"org.apache.pulsar.client.impl.schema.generic.GenericSchemaImpl",
"of", SchemaInfo.class).invoke(null, schemaInfo));
switch (schemaInfo.getType()) {
case PROTOBUF_NATIVE:
return (GenericSchema) ReflectionUtils.catchExceptions(() -> {
return (GenericSchema) ReflectionUtils.getStaticMethod(
"org.apache.pulsar.client.impl.schema.generic.GenericProtobufNativeSchema",
"of", new Class[]{SchemaInfo.class}).invoke((Object) null, schemaInfo);
});
default:
return (GenericSchema) ReflectionUtils.catchExceptions(() -> {
return (GenericSchema) ReflectionUtils.getStaticMethod(
"org.apache.pulsar.client.impl.schema.generic.GenericSchemaImpl",
"of", new Class[]{SchemaInfo.class}).invoke((Object) null, schemaInfo);

});
}

}

public static RecordSchemaBuilder newRecordSchemaBuilder(String name) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -69,6 +69,7 @@ public String getSchemaDefinition() {
case AVRO:
case JSON:
case PROTOBUF:
case PROTOBUF_NATIVE:
return new String(schema, UTF_8);
case KEY_VALUE:
KeyValue<SchemaInfo, SchemaInfo> schemaInfoKeyValue =
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -134,6 +134,11 @@ public enum SchemaType {
*/
LOCAL_DATE_TIME(19),

/**
* Protobuf native schema base on Descriptor.
*/
PROTOBUF_NATIVE(20),

//
// Schemas that don't have schema info. the value should be negative.
//
Expand Down Expand Up @@ -191,6 +196,7 @@ public static SchemaType valueOf(int value) {
case 17: return LOCAL_DATE;
case 18: return LOCAL_TIME;
case 19: return LOCAL_DATE_TIME;
case 20: return PROTOBUF_NATIVE;
case -1: return BYTES;
case -2: return AUTO;
case -3: return AUTO_CONSUME;
Expand Down Expand Up @@ -239,6 +245,7 @@ public static boolean isStructType(SchemaType type) {
case AVRO:
case JSON:
case PROTOBUF:
case PROTOBUF_NATIVE:
return true;
default:
return false;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,8 +18,6 @@
*/
package org.apache.pulsar.client.impl.schema;

import static com.google.common.base.Preconditions.checkState;

import lombok.extern.slf4j.Slf4j;
import org.apache.pulsar.client.api.Schema;
import org.apache.pulsar.client.api.SchemaSerializationException;
Expand All @@ -28,11 +26,14 @@
import org.apache.pulsar.client.api.schema.SchemaInfoProvider;
import org.apache.pulsar.client.impl.schema.generic.GenericAvroSchema;
import org.apache.pulsar.client.impl.schema.generic.GenericJsonSchema;
import org.apache.pulsar.client.impl.schema.generic.GenericProtobufNativeSchema;
import org.apache.pulsar.common.schema.KeyValue;
import org.apache.pulsar.common.schema.SchemaInfo;

import java.util.concurrent.ExecutionException;

import static com.google.common.base.Preconditions.checkState;

/**
* Auto detect schema.
*/
Expand Down Expand Up @@ -155,6 +156,8 @@ private GenericSchema generateSchema(SchemaInfo schemaInfo) {
return GenericJsonSchema.of(schemaInfo,useProvidedSchemaAsReaderSchema);
case AVRO:
return GenericAvroSchema.of(schemaInfo,useProvidedSchemaAsReaderSchema);
case PROTOBUF_NATIVE:
return GenericProtobufNativeSchema.of(schemaInfo, useProvidedSchemaAsReaderSchema);
default:
throw new IllegalArgumentException("Currently auto consume works for type '"
+ schemaInfo.getType() + "' is not supported yet");
Expand Down Expand Up @@ -199,6 +202,8 @@ public static Schema<?> getSchema(SchemaInfo schemaInfo) {
return GenericJsonSchema.of(schemaInfo);
case AVRO:
return GenericAvroSchema.of(schemaInfo);
case PROTOBUF_NATIVE:
return GenericProtobufNativeSchema.of(schemaInfo);
case KEY_VALUE:
KeyValue<SchemaInfo, SchemaInfo> kvSchemaInfo =
KeyValueSchemaInfo.decodeKeyValueSchemaInfo(schemaInfo);
Expand Down
Loading