Skip to content

Commit

Permalink
[Client] java client support UnAvroBased Schema (#8246)
Browse files Browse the repository at this point in the history
### Motivation

JAVA client support UnAvroBasedSchema

### Modifications

- `schema` division to **AvroBasedSchema** and **UnAvroBasedSchema** :

AvroBased Schema | UnAvroBased Schema
------------ | -------------
StructSchema | AbstractStructSchema
GenericSchemaImpl | AbstractGenericSchema

- `GenericSchema` add `of()` method , `GenericSchemaImpl` and `StructSchema` continue to have for backward compatibility and only support AvroBased Schema 。
  • Loading branch information
hnail authored Oct 28, 2020
1 parent 01e7cf1 commit 5955930
Show file tree
Hide file tree
Showing 10 changed files with 474 additions and 152 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@

import java.util.List;
import org.apache.pulsar.client.api.Schema;
import org.apache.pulsar.common.schema.SchemaInfo;

/**
* A schema that serializes and deserializes between {@link GenericRecord} and bytes.
Expand All @@ -40,4 +41,16 @@ public interface GenericSchema<T extends GenericRecord> extends Schema<T> {
*/
GenericRecordBuilder newRecordBuilder();


static GenericSchema of(SchemaInfo schemaInfo) {
throw new RuntimeException("GenericSchema interface implementation class must rewrite this method !");
}

static GenericSchema of(SchemaInfo schemaInfo,
boolean useProvidedSchemaAsReaderSchema) {
throw new RuntimeException("GenericSchema interface implementation class must rewrite this method !");
}



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

import com.google.common.cache.CacheBuilder;
import com.google.common.cache.CacheLoader;
import com.google.common.cache.LoadingCache;
import io.netty.buffer.ByteBuf;
import io.netty.buffer.ByteBufInputStream;
import org.apache.avro.AvroTypeException;
import org.apache.commons.codec.binary.Hex;
import org.apache.commons.lang3.SerializationException;
import org.apache.pulsar.client.api.SchemaSerializationException;
import org.apache.pulsar.client.api.schema.SchemaInfoProvider;
import org.apache.pulsar.client.api.schema.SchemaReader;
import org.apache.pulsar.client.api.schema.SchemaWriter;
import org.apache.pulsar.common.protocol.schema.BytesSchemaVersion;
import org.apache.pulsar.common.schema.SchemaInfo;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import java.util.concurrent.ExecutionException;
import java.util.concurrent.TimeUnit;

/**
* minimal abstract StructSchema
*/
public abstract class AbstractStructSchema<T> extends AbstractSchema<T> {

protected static final Logger LOG = LoggerFactory.getLogger(AbstractStructSchema.class);

protected SchemaInfo schemaInfo;
protected SchemaReader<T> reader;
protected SchemaWriter<T> writer;
protected SchemaInfoProvider schemaInfoProvider;

LoadingCache<BytesSchemaVersion, SchemaReader<T>> readerCache = CacheBuilder.newBuilder().maximumSize(100000)
.expireAfterAccess(30, TimeUnit.MINUTES).build(new CacheLoader<BytesSchemaVersion, SchemaReader<T>>() {
@Override
public SchemaReader<T> load(BytesSchemaVersion schemaVersion) {
return loadReader(schemaVersion);
}
});

public AbstractStructSchema(SchemaInfo schemaInfo){
this.schemaInfo = schemaInfo;
}


@Override
public byte[] encode(T message) {
return writer.write(message);
}

@Override
public T decode(byte[] bytes) {
return reader.read(bytes);
}

@Override
public T decode(byte[] bytes, byte[] schemaVersion) {
try {
return schemaVersion == null ? decode(bytes) :
readerCache.get(BytesSchemaVersion.of(schemaVersion)).read(bytes);
} catch (ExecutionException | AvroTypeException e) {
if (e instanceof AvroTypeException) {
throw new SchemaSerializationException(e);
}
LOG.error("Can't get generic schema for topic {} schema version {}",
schemaInfoProvider.getTopicName(), Hex.encodeHexString(schemaVersion), e);
throw new RuntimeException("Can't get generic schema for topic " + schemaInfoProvider.getTopicName());
}
}

@Override
public T decode(ByteBuf byteBuf) {
return reader.read(new ByteBufInputStream(byteBuf));
}

@Override
public T decode(ByteBuf byteBuf, byte[] schemaVersion) {
try {
return schemaVersion == null ? decode(byteBuf) :
readerCache.get(BytesSchemaVersion.of(schemaVersion)).read(new ByteBufInputStream(byteBuf));
} catch (ExecutionException e) {
LOG.error("Can't get generic schema for topic {} schema version {}",
schemaInfoProvider.getTopicName(), Hex.encodeHexString(schemaVersion), e);
throw new RuntimeException("Can't get generic schema for topic " + schemaInfoProvider.getTopicName());
}
}

@Override
public SchemaInfo getSchemaInfo() {
return this.schemaInfo;
}

@Override
public void setSchemaInfoProvider(SchemaInfoProvider schemaInfoProvider) {
this.schemaInfoProvider = schemaInfoProvider;
}

/**
* Load the schema reader for reading messages encoded by the given schema version.
*
* @param schemaVersion the provided schema version
* @return the schema reader for decoding messages encoded by the provided schema version.
*/
protected abstract SchemaReader<T> loadReader(BytesSchemaVersion schemaVersion);

/**
* TODO: think about how to make this async
*/
protected SchemaInfo getSchemaInfoByVersion(byte[] schemaVersion) {
try {
return schemaInfoProvider.getSchemaByVersion(schemaVersion).get();
} catch (InterruptedException e) {
Thread.currentThread().interrupt();
throw new SerializationException(
"Interrupted at fetching schema info for " + SchemaUtils.getStringSchemaVersion(schemaVersion),
e
);
} catch (ExecutionException e) {
throw new SerializationException(
"Failed at fetching schema info for " + SchemaUtils.getStringSchemaVersion(schemaVersion),
e.getCause()
);
}
}

protected void setWriter(SchemaWriter<T> writer) {
this.writer = writer;
}

protected void setReader(SchemaReader<T> reader) {
this.reader = reader;
}

protected SchemaReader<T> getReader() {
return reader;
}

}
Original file line number Diff line number Diff line change
Expand Up @@ -26,10 +26,10 @@
import org.apache.pulsar.client.api.schema.GenericRecord;
import org.apache.pulsar.client.api.schema.GenericSchema;
import org.apache.pulsar.client.api.schema.SchemaInfoProvider;
import org.apache.pulsar.client.impl.schema.generic.GenericSchemaImpl;
import org.apache.pulsar.client.impl.schema.generic.GenericAvroSchema;
import org.apache.pulsar.client.impl.schema.generic.GenericJsonSchema;
import org.apache.pulsar.common.schema.KeyValue;
import org.apache.pulsar.common.schema.SchemaInfo;
import org.apache.pulsar.common.schema.SchemaType;

import java.util.concurrent.ExecutionException;

Expand Down Expand Up @@ -147,13 +147,18 @@ public Schema<GenericRecord> clone() {
}

private GenericSchema generateSchema(SchemaInfo schemaInfo) {
if (schemaInfo.getType() != SchemaType.AVRO
&& schemaInfo.getType() != SchemaType.JSON) {
throw new RuntimeException("Currently auto consume only works for topics with avro or json schemas");
}
// when using `AutoConsumeSchema`, we use the schema associated with the messages as schema reader
// to decode the messages.
return GenericSchemaImpl.of(schemaInfo, false /*useProvidedSchemaAsReaderSchema*/);
final boolean useProvidedSchemaAsReaderSchema = false;
switch (schemaInfo.getType()) {
case JSON:
return GenericJsonSchema.of(schemaInfo,useProvidedSchemaAsReaderSchema);
case AVRO:
return GenericAvroSchema.of(schemaInfo,useProvidedSchemaAsReaderSchema);
default:
throw new IllegalArgumentException("Currently auto consume works for type '"
+ schemaInfo.getType() + "' is not supported yet");
}
}

public static Schema<?> getSchema(SchemaInfo schemaInfo) {
Expand Down Expand Up @@ -191,8 +196,9 @@ public static Schema<?> getSchema(SchemaInfo schemaInfo) {
case LOCAL_DATE_TIME:
return LocalDateTimeSchema.of();
case JSON:
return GenericJsonSchema.of(schemaInfo);
case AVRO:
return GenericSchemaImpl.of(schemaInfo);
return GenericAvroSchema.of(schemaInfo);
case KEY_VALUE:
KeyValue<SchemaInfo, SchemaInfo> kvSchemaInfo =
KeyValueSchemaInfo.decodeKeyValueSchemaInfo(schemaInfo);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -225,7 +225,7 @@ public CompletableFuture<SchemaInfo> getSchemaByVersion(byte[] schemaVersion) {
@Override
public CompletableFuture<SchemaInfo> getLatestSchema() {
return CompletableFuture.completedFuture(
((StructSchema<K>) keySchema).schemaInfo);
((AbstractStructSchema<K>) keySchema).schemaInfo);
}

@Override
Expand All @@ -244,7 +244,7 @@ public CompletableFuture<SchemaInfo> getSchemaByVersion(byte[] schemaVersion) {
@Override
public CompletableFuture<SchemaInfo> getLatestSchema() {
return CompletableFuture.completedFuture(
((StructSchema<V>) valueSchema).schemaInfo);
((AbstractStructSchema<V>) valueSchema).schemaInfo);
}

@Override
Expand Down
Loading

0 comments on commit 5955930

Please sign in to comment.