Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
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
Original file line number Diff line number Diff line change
Expand Up @@ -27,8 +27,12 @@
import org.apache.pulsar.common.schema.SchemaInfo;
import org.apache.pulsar.common.schema.SchemaType;

import java.io.IOException;
import java.io.ObjectInputStream;
import java.io.ObjectOutputStream;
import java.io.Serializable;
import java.util.Arrays;
import java.util.HashMap;
import java.util.Map;
import java.util.Objects;

Expand Down Expand Up @@ -57,11 +61,6 @@ public final class PulsarSchema<T> implements Serializable {
private transient Schema<T> schema;
private transient SchemaInfo schemaInfo;

private String schemaName;
private byte[] schemaBytes;
private SchemaType schemaType;
private Map<String, String> schemaProperties;

/** Create serializable pulsar schema for primitive types. */
public PulsarSchema(Schema<T> schema) {
SchemaInfo info = schema.getSchemaInfo();
Expand All @@ -78,7 +77,8 @@ public PulsarSchema(Schema<T> schema) {
// Primitive type information could be reflected from the schema class.
Class<?> typeClass = getTemplateType1(schema.getClass());

setSchemaInfo(encodeClassInfo(info, typeClass));
this.schemaInfo = encodeClassInfo(info, typeClass);
this.schema = createSchema(schemaInfo);
}

/**
Expand All @@ -94,7 +94,8 @@ public PulsarSchema(Schema<T> schema, Class<T> typeClass) {
"Key Value Schema should provide the type classes of key and value");
validateSchemaInfo(info);

setSchemaInfo(encodeClassInfo(info, typeClass));
this.schemaInfo = encodeClassInfo(info, typeClass);
this.schema = createSchema(schemaInfo);
}

/** Create serializable pulsar schema for key value type. */
Expand All @@ -117,37 +118,66 @@ public <K, V> PulsarSchema(
SchemaInfo encodedInfo =
encodeKeyValueSchemaInfo(info.getName(), infoKey, infoValue, encodingType);

setSchemaInfo(encodeClassInfo(encodedInfo, KeyValue.class));
}

/** Validate the schema for having the required class info. */
private void setSchemaInfo(SchemaInfo schemaInfo) {
this.schema = createSchema(schemaInfo);
this.schemaInfo = schemaInfo;

this.schemaName = schemaInfo.getName();
this.schemaBytes = schemaInfo.getSchema();
this.schemaType = schemaInfo.getType();
this.schemaProperties = schemaInfo.getProperties();
this.schemaInfo = encodeClassInfo(encodedInfo, KeyValue.class);
this.schema = createSchema(this.schemaInfo);
}

public Schema<T> getPulsarSchema() {
if (schema == null) {
this.schema = createSchema(getSchemaInfo());
}
return schema;
}

public SchemaInfo getSchemaInfo() {
if (schemaInfo == null) {
this.schemaInfo =
new SchemaInfoImpl(schemaName, schemaBytes, schemaType, schemaProperties);
}
return schemaInfo;
}

public Class<T> getRecordClass() {
return decodeClassInfo(getSchemaInfo());
return decodeClassInfo(schemaInfo);
}

private void writeObject(ObjectOutputStream oos) throws IOException {
// Name
oos.writeUTF(schemaInfo.getName());

// Schema
byte[] schemaBytes = schemaInfo.getSchema();
oos.writeInt(schemaBytes.length);
oos.write(schemaBytes);

// Type
SchemaType type = schemaInfo.getType();
oos.writeInt(type.getValue());

// Properties
Map<String, String> properties = schemaInfo.getProperties();
oos.writeInt(properties.size());
for (Map.Entry<String, String> entry : properties.entrySet()) {
oos.writeUTF(entry.getKey());
oos.writeUTF(entry.getValue());
}
}

private void readObject(ObjectInputStream ois) throws ClassNotFoundException, IOException {
// Name
String name = ois.readUTF();

// Schema
int byteLen = ois.readInt();
byte[] schemaBytes = new byte[byteLen];
ois.readFully(schemaBytes);

// Type
int typeIdx = ois.readInt();
SchemaType type = SchemaType.valueOf(typeIdx);

// Properties
int propSize = ois.readInt();
Map<String, String> properties = new HashMap<>(propSize);
for (int i = 0; i < propSize; i++) {
properties.put(ois.readUTF(), ois.readUTF());
}

this.schemaInfo = new SchemaInfoImpl(name, schemaBytes, type, properties);
this.schema = createSchema(schemaInfo);
}

@Override
Expand All @@ -158,21 +188,24 @@ public boolean equals(Object o) {
if (o == null || getClass() != o.getClass()) {
return false;
}
PulsarSchema<?> that = ((PulsarSchema<?>) o);
SchemaInfo that = ((PulsarSchema<?>) o).getPulsarSchema().getSchemaInfo();

return Objects.equals(schemaType, that.schemaType)
&& Arrays.equals(schemaBytes, that.schemaBytes)
&& Objects.equals(schemaProperties, that.schemaProperties);
return Objects.equals(schemaInfo.getType(), that.getType())
&& Arrays.equals(schemaInfo.getSchema(), that.getSchema())
&& Objects.equals(schemaInfo.getProperties(), that.getProperties());
}

@Override
public int hashCode() {
return Objects.hash(schemaType, Arrays.hashCode(schemaBytes), schemaProperties);
return Objects.hash(
schemaInfo.getType(),
Arrays.hashCode(schemaInfo.getSchema()),
schemaInfo.getProperties());
}

@Override
public String toString() {
return getSchemaInfo().toString();
return schemaInfo.toString();
}

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -50,10 +50,7 @@ public static void serializeBytes(DataOutputStream out, byte[] bytes) throws IOE
public static byte[] deserializeBytes(DataInputStream in) throws IOException {
int size = in.readInt();
byte[] bytes = new byte[size];
int result = in.read(bytes);
if (result < 0) {
throw new IOException("Couldn't deserialize the object, wrong byte buffer.");
}
in.readFully(bytes);

return bytes;
}
Expand Down