Skip to content

Commit

Permalink
Implement GenericObject - Allow GenericRecord to wrap any Java Object (
Browse files Browse the repository at this point in the history
…apache#10057)

Contents:
- introduce new high level interface GenericObject, that represents every logical value on the Pulsar topic
- allow AutoConsumeSchema to deal with every Schema type
- handle schema less topics with AutoConsumeSchema (return Bytes schema and byte[] payload)
- rename `getNativeRecord` to `getNativeObject`

How it works:
- in case of non struct schema, like for primitives and for KeyValue, wrap the result into a PrimitiveRecord class, that is an implementation of GenericRecord that wraps a Object and a SchemaType, it returns an empty list of "fields"

Intended audience:
- allow users to implement Sink<GenericObject>, that is to write Sinks that work with every Schema type (the patch for Pulsar IO will follow up as a separate PR)

```
interface GenericObject {
      Object getNativeObject();
      SchemaType getSchemaType();
}
interface GenericRecord extends GenericObject
```
  • Loading branch information
eolivelli committed May 12, 2021
1 parent 6fe93f5 commit 23b2859
Show file tree
Hide file tree
Showing 12 changed files with 483 additions and 34 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -3883,6 +3883,55 @@ public void testSendCompressedWithDeferredSchemaSetup(boolean enableBatching) th

GenericRecord res = consumer.receive().getValue();
consumer.close();
assertEquals(SchemaType.AVRO, res.getSchemaType());
org.apache.avro.generic.GenericRecord nativeRecord = (org.apache.avro.generic.GenericRecord) res.getNativeObject();
org.apache.avro.Schema schema = nativeRecord.getSchema();
for (org.apache.pulsar.client.api.schema.Field f : res.getFields()) {
log.info("field {} {}", f.getName(), res.getField(f));
assertEquals("field", f.getName());
assertEquals("aaaaaaaaaaaaaaaaaaaaaaaaa", res.getField(f));
assertEquals("aaaaaaaaaaaaaaaaaaaaaaaaa", nativeRecord.get(f.getName()).toString());
}

assertEquals(1, res.getFields().size());
}

@DataProvider(name = "avroSchemaProvider")
public static Object[] avroSchemaProvider() {
return new Object[]{Schema.AVRO(MyBean.class), Schema.JSON(MyBean.class)};
}

@Test(dataProvider = "avroSchemaProvider")
public void testAccessAvroSchemaMetadata(Schema<MyBean> schema) throws Exception {
log.info("-- Starting {} test --", methodName);

final String topic = "persistent://my-property/my-ns/accessSchema";
Consumer<GenericRecord> consumer = pulsarClient.newConsumer(Schema.AUTO_CONSUME())
.topic(topic)
.subscriptionName("testsub")
.subscribe();

Producer<MyBean> producer = pulsarClient
.newProducer(schema)
.topic(topic)
.create();
MyBean payload = new MyBean();
payload.setField("aaaaaaaaaaaaaaaaaaaaaaaaa");
producer.send(payload);
producer.close();

GenericRecord res = consumer.receive().getValue();
consumer.close();
assertEquals(schema.getSchemaInfo().getType(), res.getSchemaType());
org.apache.avro.generic.GenericRecord nativeAvroRecord = null;
JsonNode nativeJsonRecord = null;
if (schema.getSchemaInfo().getType() == SchemaType.AVRO) {
nativeAvroRecord = (org.apache.avro.generic.GenericRecord) res.getNativeObject();
assertNotNull(nativeAvroRecord);
} else {
nativeJsonRecord = (JsonNode) res.getNativeObject();
assertNotNull(nativeJsonRecord);
}
for (org.apache.pulsar.client.api.schema.Field f : res.getFields()) {
log.info("field {} {}", f.getName(), res.getField(f));
assertEquals("field", f.getName());
Expand Down
101 changes: 101 additions & 0 deletions pulsar-broker/src/test/java/org/apache/pulsar/schema/SchemaTest.java
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,7 @@
import static org.testng.Assert.assertTrue;
import static org.testng.Assert.fail;
import com.google.common.collect.Sets;
import java.nio.charset.StandardCharsets;
import java.util.Collections;
import java.util.List;
import java.util.concurrent.CompletableFuture;
Expand Down Expand Up @@ -203,6 +204,106 @@ public void testBytesSchemaDeserialize() throws Exception {
consumer1.close();
}

@Test
public void testStringSchema() throws Exception {
final String tenant = PUBLIC_TENANT;
final String namespace = "test-namespace-" + randomName(16);
final String topicName = "test-string-schema";

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

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

admin.topics().createPartitionedTopic(topic, 2);

Producer<String> producer = pulsarClient
.newProducer(Schema.STRING)
.topic(topic)
.create();

Consumer<String> consumer = pulsarClient.newConsumer(Schema.STRING)
.subscriptionName("test-sub")
.topic(topic)
.subscribe();

// use GenericRecord even for primitive types
// it will be a PrimitiveRecord
Consumer<GenericRecord> consumer2 = pulsarClient.newConsumer(Schema.AUTO_CONSUME())
.subscriptionName("test-sub3")
.topic(topic)
.subscribe();

producer.send("foo");

Message<String> message = consumer.receive();
Message<GenericRecord> message3 = consumer2.receive();

assertEquals("foo", message.getValue());
assertEquals(message3.getValue().getClass().getName(), "org.apache.pulsar.client.impl.schema.GenericObjectWrapper");
assertEquals(SchemaType.STRING, message3.getValue().getSchemaType());
assertEquals("foo", message3.getValue().getNativeObject());

producer.close();
consumer.close();
consumer2.close();
}

@Test
public void testUseAutoConsumeWithSchemalessTopic() throws Exception {
final String tenant = PUBLIC_TENANT;
final String namespace = "test-namespace-" + randomName(16);
final String topicName = "test-schemaless";

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

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

admin.topics().createPartitionedTopic(topic, 2);

Producer<byte[]> producer = pulsarClient
.newProducer()
.topic(topic)
.create();

Consumer<byte[]> consumer = pulsarClient.newConsumer()
.subscriptionName("test-sub")
.topic(topic)
.subscribe();

// use GenericRecord even for primitive types
// it will be a PrimitiveRecord
Consumer<GenericRecord> consumer2 = pulsarClient.newConsumer(Schema.AUTO_CONSUME())
.subscriptionName("test-sub3")
.topic(topic)
.subscribe();

producer.send("foo".getBytes(StandardCharsets.UTF_8));

Message<byte[]> message = consumer.receive();
Message<GenericRecord> message3 = consumer2.receive();

assertEquals("foo".getBytes(StandardCharsets.UTF_8), message.getValue());
assertEquals(message3.getValue().getClass().getName(), "org.apache.pulsar.client.impl.schema.GenericObjectWrapper");
assertEquals(SchemaType.BYTES, message3.getValue().getSchemaType());
assertEquals("foo".getBytes(StandardCharsets.UTF_8), message3.getValue().getNativeObject());

producer.close();
consumer.close();
consumer2.close();
}

@Test
public void testIsUsingAvroSchemaParser() {
for (SchemaType value : SchemaType.values()) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -28,6 +28,7 @@
import java.util.Date;
import org.apache.pulsar.client.api.schema.GenericRecord;
import org.apache.pulsar.client.api.schema.GenericSchema;
import org.apache.pulsar.client.api.schema.GenericObject;
import org.apache.pulsar.client.api.schema.SchemaDefinition;
import org.apache.pulsar.client.api.schema.SchemaInfoProvider;
import org.apache.pulsar.client.internal.DefaultImplementation;
Expand Down Expand Up @@ -361,9 +362,8 @@ static Schema<GenericRecord> AUTO() {
* Create a schema instance that automatically deserialize messages
* based on the current topic schema.
*
* <p>The messages values are deserialized into a {@link GenericRecord} object.
*
* <p>Currently this is only supported with Avro and JSON schema types.
* <p>The messages values are deserialized into a {@link GenericRecord} object,
* that extends the {@link GenericObject} interface.
*
* @return the auto schema instance
*/
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.client.api.schema;

import org.apache.pulsar.common.schema.SchemaType;

/**
* This is an abstraction over the logical value that is store into a Message.
* Pulsar decodes the payload of the Message using the Schema that is configured for the topic.
*/
public interface GenericObject {

/**
* Return the schema tyoe.
*
* @return the schema type
* @throws UnsupportedOperationException if this feature is not implemented
* @see SchemaType#BYTES when the topic has no schema information
* @see SchemaType#STRING
* @see SchemaType#AVRO
* @see SchemaType#PROTOBUF_NATIVE
* @see SchemaType#JSON
*/
SchemaType getSchemaType();

/**
* Return the internal native representation of the Object,
* like a AVRO GenericRecord, a String or a byte[].
*
* @return the decoded object
* @throws UnsupportedOperationException if the operation is not supported
*/
Object getNativeObject();
}
Original file line number Diff line number Diff line change
Expand Up @@ -27,7 +27,7 @@
*/
@InterfaceAudience.Public
@InterfaceStability.Stable
public interface GenericRecord {
public interface GenericRecord extends GenericObject {

/**
* Return schema version.
Expand Down Expand Up @@ -61,4 +61,30 @@ default Object getField(Field field) {
*/
Object getField(String fieldName);

/**
* Return the schema tyoe.
*
* @return the schema type
* @throws UnsupportedOperationException if this feature is not implemented
* @see SchemaType#AVRO
* @see SchemaType#PROTOBUF_NATIVE
* @see SchemaType#JSON
*/
@Override
default SchemaType getSchemaType() {
throw new UnsupportedOperationException();
}

/**
* Return the internal native representation of the Record,
* like a AVRO GenericRecord.
*
* @return the internal representation of the record
* @throws UnsupportedOperationException if the operation is not supported
*/
@Override
default Object getNativeObject() {
throw new UnsupportedOperationException();
}

}
Loading

0 comments on commit 23b2859

Please sign in to comment.