Skip to content

Commit

Permalink
KeyValue schema support for pulsar sql (apache#6325)
Browse files Browse the repository at this point in the history
Fixes apache#5560

### Motivation

Currently, Pulsar SQL can't read the keyValue schema data. This PR added support Pulsar SQL reading messages with a key-value schema.

### Modifications

Add KeyValue schema support for Pulsar SQL. Add prefix __key. for the key field name.
  • Loading branch information
gaoran10 authored Feb 17, 2020
1 parent df15210 commit 3cf6be1
Show file tree
Hide file tree
Showing 18 changed files with 705 additions and 72 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -101,4 +101,19 @@ public interface RawMessage {
* @return the key of the message
*/
Optional<String> getKey();

/**
* Get byteBuf of the key.
*
* @return the byte array with the key payload
*/
Optional<ByteBuf> getKeyBytes();

/**
* Check whether the key has been base64 encoded.
*
* @return true if the key is base64 encoded, false otherwise
*/
boolean hasBase64EncodedKey();

}
Original file line number Diff line number Diff line change
Expand Up @@ -19,9 +19,10 @@
package org.apache.pulsar.common.api.raw;

import io.netty.buffer.ByteBuf;
import io.netty.buffer.Unpooled;
import io.netty.util.Recycler;
import io.netty.util.Recycler.Handle;

import java.util.Base64;
import java.util.Collections;
import java.util.Map;
import java.util.Optional;
Expand Down Expand Up @@ -140,4 +141,25 @@ public Optional<String> getKey() {
return Optional.empty();
}
}

@Override
public Optional<ByteBuf> getKeyBytes() {
if (getKey().isPresent()) {
if (hasBase64EncodedKey()) {
return Optional.of(Unpooled.wrappedBuffer(Base64.getDecoder().decode(getKey().get())));
} else {
return Optional.of(Unpooled.wrappedBuffer(getKey().get().getBytes()));
}
}
return Optional.empty();
}

@Override
public boolean hasBase64EncodedKey() {
if (singleMessageMetadata != null) {
return singleMessageMetadata.getPartitionKeyB64Encoded();
}
return msgMetadata.get().getPartitionKeyB64Encoded();
}

}
Original file line number Diff line number Diff line change
Expand Up @@ -77,6 +77,11 @@ public Object deserialize(ByteBuf payload) {
return null;
}

@Override
public Object deserialize(ByteBuf keyPayload, ByteBuf dataPayload) {
return null;
}

@Override
public Object extractField(int index, Object currentRecord) {
try {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -73,6 +73,11 @@ public Object deserialize(ByteBuf payload) {
}
}

@Override
public Object deserialize(ByteBuf keyPayload, ByteBuf dataPayload) {
return null;
}

@Override
public Object extractField(int index, Object currentRecord) {
try {
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,91 @@
/**
* 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.sql.presto;

import io.airlift.log.Logger;
import io.netty.buffer.ByteBuf;
import java.util.List;
import java.util.Objects;
import org.apache.pulsar.client.impl.schema.KeyValueSchemaInfo;
import org.apache.pulsar.common.schema.KeyValue;
import org.apache.pulsar.common.schema.KeyValueEncodingType;
import org.apache.pulsar.common.schema.SchemaInfo;


/**
* Schema handler for payload in the KeyValue format.
*/
public class KeyValueSchemaHandler implements SchemaHandler {

private static final Logger log = Logger.get(KeyValueSchemaHandler.class);

private final List<PulsarColumnHandle> columnHandles;

private final SchemaHandler keySchemaHandler;

private final SchemaHandler valueSchemaHandler;

private KeyValueEncodingType keyValueEncodingType;

public KeyValueSchemaHandler(SchemaInfo schemaInfo, List<PulsarColumnHandle> columnHandles) {
this.columnHandles = columnHandles;
KeyValue<SchemaInfo, SchemaInfo> kvSchemaInfo = KeyValueSchemaInfo.decodeKeyValueSchemaInfo(schemaInfo);
keySchemaHandler = PulsarSchemaHandlers.newPulsarSchemaHandler(kvSchemaInfo.getKey(), columnHandles);
valueSchemaHandler = PulsarSchemaHandlers.newPulsarSchemaHandler(kvSchemaInfo.getValue(), columnHandles);
keyValueEncodingType = KeyValueSchemaInfo.decodeKeyValueEncodingType(schemaInfo);
}

@Override
public Object deserialize(ByteBuf payload) {
return null;
}

@Override
public Object deserialize(ByteBuf keyPayload, ByteBuf dataPayload) {
ByteBuf keyByteBuf;
ByteBuf valueByteBuf;
if (Objects.equals(keyValueEncodingType, KeyValueEncodingType.INLINE)) {
dataPayload.resetReaderIndex();
int keyLength = dataPayload.readInt();
keyByteBuf = dataPayload.readSlice(keyLength);

int valueLength = dataPayload.readInt();
valueByteBuf = dataPayload.readSlice(valueLength);
} else {
keyByteBuf = keyPayload;
valueByteBuf = dataPayload;
}
Object keyObj = keySchemaHandler.deserialize(keyByteBuf);
Object valueObj = valueSchemaHandler.deserialize(valueByteBuf);
return new KeyValue<>(keyObj, valueObj);
}

@Override
public Object extractField(int index, Object currentRecord) {
PulsarColumnHandle pulsarColumnHandle = this.columnHandles.get(index);
KeyValue<Object, Object> keyValue = (KeyValue<Object, Object>) currentRecord;
if (pulsarColumnHandle.isKey()) {
return keySchemaHandler.extractField(index, keyValue.getKey());
} else if (pulsarColumnHandle.isValue()) {
return valueSchemaHandler.extractField(index, keyValue.getValue());
}
return null;
}

}
Original file line number Diff line number Diff line change
Expand Up @@ -24,8 +24,10 @@
import com.facebook.presto.spi.ColumnMetadata;
import com.facebook.presto.spi.type.Type;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonIgnore;
import com.fasterxml.jackson.annotation.JsonProperty;
import java.util.Arrays;
import java.util.Objects;

/**
* This class represents the basic information about a presto column.
Expand Down Expand Up @@ -58,6 +60,26 @@ public class PulsarColumnHandle implements ColumnHandle {

private final Integer[] positionIndices;

private HandleKeyValueType handleKeyValueType;

/**
* Column Handle keyValue type, used for keyValue schema.
*/
public enum HandleKeyValueType {
/**
* The handle not for keyValue schema.
*/
NONE,
/**
* The key schema handle for keyValue schema.
*/
KEY,
/**
* The value schema handle for keyValue schema.
*/
VALUE
}

@JsonCreator
public PulsarColumnHandle(
@JsonProperty("connectorId") String connectorId,
Expand All @@ -66,14 +88,20 @@ public PulsarColumnHandle(
@JsonProperty("hidden") boolean hidden,
@JsonProperty("internal") boolean internal,
@JsonProperty("fieldNames") String[] fieldNames,
@JsonProperty("positionIndices") Integer[] positionIndices) {
@JsonProperty("positionIndices") Integer[] positionIndices,
@JsonProperty("handleKeyValueType") HandleKeyValueType handleKeyValueType) {
this.connectorId = requireNonNull(connectorId, "connectorId is null");
this.name = requireNonNull(name, "name is null");
this.type = requireNonNull(type, "type is null");
this.hidden = hidden;
this.internal = internal;
this.fieldNames = fieldNames;
this.positionIndices = positionIndices;
if (handleKeyValueType == null) {
this.handleKeyValueType = HandleKeyValueType.NONE;
} else {
this.handleKeyValueType = handleKeyValueType;
}
}

@JsonProperty
Expand Down Expand Up @@ -111,6 +139,20 @@ public Integer[] getPositionIndices() {
return positionIndices;
}

@JsonProperty
public HandleKeyValueType getHandleKeyValueType() {
return handleKeyValueType;
}

@JsonIgnore
public boolean isKey() {
return Objects.equals(handleKeyValueType, HandleKeyValueType.KEY);
}

@JsonIgnore
public boolean isValue() {
return Objects.equals(handleKeyValueType, HandleKeyValueType.VALUE);
}

ColumnMetadata getColumnMetadata() {
return new ColumnMetadata(name, type, null, hidden);
Expand Down Expand Up @@ -145,7 +187,10 @@ public boolean equals(Object o) {
if (!Arrays.deepEquals(fieldNames, that.fieldNames)) {
return false;
}
return Arrays.deepEquals(positionIndices, that.positionIndices);
if (!Arrays.deepEquals(positionIndices, that.positionIndices)) {
return false;
}
return Objects.equals(handleKeyValueType, that.handleKeyValueType);
}

@Override
Expand All @@ -157,6 +202,7 @@ public int hashCode() {
result = 31 * result + (internal ? 1 : 0);
result = 31 * result + Arrays.hashCode(fieldNames);
result = 31 * result + Arrays.hashCode(positionIndices);
result = 31 * result + (handleKeyValueType != null ? handleKeyValueType.hashCode() : 0);
return result;
}

Expand All @@ -170,6 +216,7 @@ public String toString() {
+ ", internal=" + internal
+ ", fieldNames=" + Arrays.toString(fieldNames)
+ ", positionIndices=" + Arrays.toString(positionIndices)
+ ", handleKeyValueType=" + handleKeyValueType
+ '}';
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@
import com.facebook.presto.spi.ColumnMetadata;
import com.facebook.presto.spi.type.Type;
import java.util.Arrays;
import java.util.Objects;

/**
* Description of the column metadata.
Expand All @@ -32,15 +33,19 @@ public class PulsarColumnMetadata extends ColumnMetadata {
private String nameWithCase;
private String[] fieldNames;
private Integer[] positionIndices;
private PulsarColumnHandle.HandleKeyValueType handleKeyValueType;
public final static String KEY_SCHEMA_COLUMN_PREFIX = "__key.";

public PulsarColumnMetadata(String name, Type type, String comment, String extraInfo,
boolean hidden, boolean isInternal,
String[] fieldNames, Integer[] positionIndices) {
String[] fieldNames, Integer[] positionIndices,
PulsarColumnHandle.HandleKeyValueType handleKeyValueType) {
super(name, type, comment, extraInfo, hidden);
this.nameWithCase = name;
this.isInternal = isInternal;
this.fieldNames = fieldNames;
this.positionIndices = positionIndices;
this.handleKeyValueType = handleKeyValueType;
}

public String getNameWithCase() {
Expand All @@ -59,13 +64,33 @@ public Integer[] getPositionIndices() {
return positionIndices;
}

public PulsarColumnHandle.HandleKeyValueType getHandleKeyValueType() {
return handleKeyValueType;
}

public boolean isKey() {
return Objects.equals(handleKeyValueType, PulsarColumnHandle.HandleKeyValueType.KEY);
}

public boolean isValue() {
return Objects.equals(handleKeyValueType, PulsarColumnHandle.HandleKeyValueType.VALUE);
}

public static String getColumnName(PulsarColumnHandle.HandleKeyValueType handleKeyValueType, String name) {
if (Objects.equals(PulsarColumnHandle.HandleKeyValueType.KEY, handleKeyValueType)) {
return KEY_SCHEMA_COLUMN_PREFIX + name;
}
return name;
}

@Override
public String toString() {
return "PulsarColumnMetadata{"
+ "isInternal=" + isInternal
+ ", nameWithCase='" + nameWithCase + '\''
+ ", fieldNames=" + Arrays.toString(fieldNames)
+ ", positionIndices=" + Arrays.toString(positionIndices)
+ ", handleKeyValueType=" + handleKeyValueType
+ '}';
}

Expand All @@ -92,7 +117,10 @@ public boolean equals(Object o) {
if (!Arrays.deepEquals(fieldNames, that.fieldNames)) {
return false;
}
return Arrays.deepEquals(positionIndices, that.positionIndices);
if (!Arrays.deepEquals(positionIndices, that.positionIndices)) {
return false;
}
return Objects.equals(handleKeyValueType, that.handleKeyValueType);
}

@Override
Expand All @@ -102,6 +130,7 @@ public int hashCode() {
result = 31 * result + (nameWithCase != null ? nameWithCase.hashCode() : 0);
result = 31 * result + Arrays.hashCode(fieldNames);
result = 31 * result + Arrays.hashCode(positionIndices);
result = 31 * result + (handleKeyValueType != null ? handleKeyValueType.hashCode() : 0);
return result;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -218,11 +218,12 @@ PulsarColumnHandle getColumnHandle(String connectorId, boolean hidden) {
getName(),
getType(),
hidden,
true, null, null);
true, null, null, PulsarColumnHandle.HandleKeyValueType.NONE);
}

PulsarColumnMetadata getColumnMetadata(boolean hidden) {
return new PulsarColumnMetadata(name, type, comment, null, hidden, true, null, null);
return new PulsarColumnMetadata(name, type, comment, null, hidden, true, null, null,
PulsarColumnHandle.HandleKeyValueType.NONE);
}

public static Set<PulsarInternalColumn> getInternalFields() {
Expand Down
Loading

0 comments on commit 3cf6be1

Please sign in to comment.