-
Notifications
You must be signed in to change notification settings - Fork 3.5k
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
Support multiple Avro schema version in Pulsar SQL #4847
Changes from 1 commit
0e0e9a0
5ba4030
277ecfb
b87e142
244d22f
1771681
137e1be
235048a
123f5d8
e06f6be
1779434
f8c11f0
78e86b5
78df48c
910b127
57132c4
51239bd
bfce092
59e2c40
1cac019
13ecb9a
c9bd00f
47aaccd
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -18,86 +18,101 @@ | |
*/ | ||
package org.apache.pulsar.sql.presto; | ||
|
||
import com.google.common.annotations.VisibleForTesting; | ||
import io.airlift.log.Logger; | ||
|
||
import io.netty.buffer.ByteBuf; | ||
import io.netty.buffer.ByteBufAllocator; | ||
import io.netty.util.ReferenceCountUtil; | ||
import io.netty.util.concurrent.FastThreadLocal; | ||
import org.apache.avro.Schema; | ||
import org.apache.avro.generic.GenericDatumReader; | ||
import org.apache.avro.generic.GenericRecord; | ||
import org.apache.avro.io.BinaryDecoder; | ||
import org.apache.avro.io.DatumReader; | ||
import org.apache.avro.io.DecoderFactory; | ||
|
||
import java.io.IOException; | ||
|
||
import org.apache.pulsar.client.api.PulsarClientException; | ||
import org.apache.pulsar.client.impl.schema.generic.GenericAvroRecord; | ||
import org.apache.pulsar.client.impl.schema.generic.GenericAvroSchema; | ||
import org.apache.pulsar.common.api.raw.RawMessage; | ||
import org.apache.pulsar.common.naming.TopicName; | ||
import org.apache.pulsar.common.schema.SchemaInfo; | ||
|
||
import java.util.List; | ||
|
||
public class AvroSchemaHandler implements SchemaHandler { | ||
|
||
private final DatumReader<GenericRecord> datumReader; | ||
|
||
private final List<PulsarColumnHandle> columnHandles; | ||
|
||
private static final FastThreadLocal<BinaryDecoder> decoders = | ||
new FastThreadLocal<>(); | ||
private final GenericAvroSchema genericAvroSchema; | ||
|
||
private final SchemaInfo schemaInfo; | ||
|
||
private static final FastThreadLocal<byte[]> tmpBuffer = new FastThreadLocal<byte[]>() { | ||
@Override | ||
protected byte[] initialValue() { | ||
return new byte[1024]; | ||
} | ||
}; | ||
|
||
private static final Logger log = Logger.get(AvroSchemaHandler.class); | ||
|
||
public AvroSchemaHandler(Schema schema, List<PulsarColumnHandle> columnHandles) { | ||
this.datumReader = new GenericDatumReader<>(schema); | ||
public AvroSchemaHandler(TopicName topicName, PulsarConnectorConfig pulsarConnectorConfig, | ||
SchemaInfo schemaInfo, List<PulsarColumnHandle> columnHandles) throws PulsarClientException { | ||
this.schemaInfo = schemaInfo; | ||
this.genericAvroSchema = new GenericAvroSchema(schemaInfo); | ||
this.genericAvroSchema | ||
.setSchemaInfoProvider(new PulsarSqlSchemaInfoProvider(topicName, pulsarConnectorConfig.getPulsarAdmin())); | ||
this.columnHandles = columnHandles; | ||
} | ||
|
||
@Override | ||
public Object deserialize(ByteBuf payload) { | ||
|
||
ByteBuf heapBuffer = null; | ||
try { | ||
BinaryDecoder decoderFromCache = decoders.get(); | ||
|
||
// Make a copy into a heap buffer, since Avro cannot deserialize directly from direct memory | ||
int size = payload.readableBytes(); | ||
heapBuffer = ByteBufAllocator.DEFAULT.heapBuffer(size, size); | ||
heapBuffer.writeBytes(payload); | ||
AvroSchemaHandler(PulsarSqlSchemaInfoProvider pulsarSqlSchemaInfoProvider, SchemaInfo schemaInfo, List<PulsarColumnHandle> columnHandles) { | ||
this.schemaInfo = schemaInfo; | ||
this.genericAvroSchema = new GenericAvroSchema(schemaInfo); | ||
this.genericAvroSchema.setSchemaInfoProvider(pulsarSqlSchemaInfoProvider); | ||
this.columnHandles = columnHandles; | ||
} | ||
|
||
BinaryDecoder decoder = DecoderFactory.get().binaryDecoder(heapBuffer.array(), heapBuffer.arrayOffset(), | ||
heapBuffer.readableBytes(), decoderFromCache); | ||
if (decoderFromCache==null) { | ||
decoders.set(decoder); | ||
} | ||
return this.datumReader.read(null, decoder); | ||
} catch (IOException e) { | ||
log.error(e); | ||
} finally { | ||
ReferenceCountUtil.safeRelease(heapBuffer); | ||
@Override | ||
public Object deserialize(RawMessage rawMessage) { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. You can just add a method deserialize(ByteBuf byteBuf, byte[] schemaVersion) to instead use RawMessage as an input param. |
||
ByteBuf payload = rawMessage.getData(); | ||
int size = payload.readableBytes(); | ||
byte[] buffer = tmpBuffer.get(); | ||
if (buffer.length < size) { | ||
// If the thread-local buffer is not big enough, replace it with | ||
// a bigger one | ||
buffer = new byte[size * 2]; | ||
tmpBuffer.set(buffer); | ||
} | ||
payload.readBytes(buffer, 0, size); | ||
if (rawMessage.getSchemaVersion() != null) { | ||
return genericAvroSchema.decode(buffer, rawMessage.getSchemaVersion()); | ||
} else { | ||
return genericAvroSchema.decode(buffer); | ||
} | ||
return null; | ||
} | ||
|
||
@Override | ||
public Object extractField(int index, Object currentRecord) { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I don't quite understand the reason for the change in this method. I think to support multiple schema version decode does not affect extract field from the GenericRecord right? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I only add a deserialize method, and add a default interface for the keyPayload deserialize. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I'm talking about |
||
try { | ||
GenericRecord record = (GenericRecord) currentRecord; | ||
GenericAvroRecord record = (GenericAvroRecord) currentRecord; | ||
PulsarColumnHandle pulsarColumnHandle = this.columnHandles.get(index); | ||
Integer[] positionIndices = pulsarColumnHandle.getPositionIndices(); | ||
Object curr = record.get(positionIndices[0]); | ||
if (curr == null) { | ||
return null; | ||
} | ||
if (positionIndices.length > 0) { | ||
for (int i = 1 ; i < positionIndices.length; i++) { | ||
curr = ((GenericRecord) curr).get(positionIndices[i]); | ||
if (curr == null) { | ||
return null; | ||
} | ||
String[] names = pulsarColumnHandle.getFieldNames(); | ||
|
||
if (names.length == 1) { | ||
return record.getField(pulsarColumnHandle.getName()); | ||
} else { | ||
for (int i = 0 ; i < names.length - 1; i++) { | ||
record = (GenericAvroRecord) record.getField(names[i]); | ||
} | ||
return record.getField(names[names.length - 1]); | ||
} | ||
return curr; | ||
} catch (Exception ex) { | ||
log.debug(ex,"%s", ex); | ||
} | ||
return null; | ||
} | ||
|
||
@VisibleForTesting | ||
GenericAvroSchema getSchema() { | ||
return this.genericAvroSchema; | ||
} | ||
|
||
@VisibleForTesting | ||
SchemaInfo getSchemaInfo() { | ||
return schemaInfo; | ||
} | ||
} |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
You can get schema version by msgMetadata directly.