-
Notifications
You must be signed in to change notification settings - Fork 3.6k
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
KeyValue schema support for pulsar sql #6325
Changes from 5 commits
f29d602
dd44892
fcd5fa6
0a5f532
1057706
be97bf2
35a7056
e2a9226
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 |
---|---|---|
@@ -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 |
---|---|---|
|
@@ -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. | ||
|
@@ -32,15 +33,20 @@ 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 final static String VALUE_SCHEMA_COLUMN_PREFIX = "value."; | ||
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 think we need to add 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. Yes, it’s better! 👍 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. Maybe we could use 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. Yes. +1 |
||
|
||
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() { | ||
|
@@ -59,13 +65,35 @@ 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; | ||
} else if (Objects.equals(PulsarColumnHandle.HandleKeyValueType.VALUE, handleKeyValueType)) { | ||
return VALUE_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 | ||
+ '}'; | ||
} | ||
|
||
|
@@ -92,7 +120,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 | ||
|
@@ -102,6 +133,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; | ||
} | ||
} |
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.
It's better to named isKeyBase64Encoded().