From de1b8b3e7865d352f19ca12348af15368df4df38 Mon Sep 17 00:00:00 2001 From: hnail Date: Mon, 1 Feb 2021 22:55:41 +0800 Subject: [PATCH] [PIP-71][SQL]Pulsar SQL migrate SchemaHandle to presto decoder (#8422) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Fixes #4747 Fixes #7652 ### Motivation PIP-71: https://github.com/apache/pulsar/wiki/PIP-71:-Pulsar-SQL-migrate-SchemaHandle-to-presto-decoder **Pip-Doc** : [[PIP-71][SQL]Migrate SchemaHandle to Presto-decoder](https://docs.google.com/document/d/1KwG0GoHccju4-QNPfvT6tOwhp5Fvs6-iZlfLooPxTDM/edit?usp=sharing) In current version , pulsar-presto deserialize fields rely on SchemaHandler , but this causes the following restrictions : - **Metadata**: current nested field is dissociate with presto ParameterizedType , It treated nested field as a separated field , so presto compiler can’t understand the type hierarchy . nested field should be Row type in presto (e.g. Hive struct type support) . In the same way,array \ map type also shoud associate with presto ParameterizedTypes. - **Decoder** : SchemaHandler is hard to work with `RecordCursor.getObject()` to support ROW,MAP,ARRAY .etc The **motivations** of this pull request : - ` PulsarMetadata` take advantage of `ParameterizedType` to describe `row/array/map` Type instead of resolve nested columns in pulsar-presto connecter. - Customize `RowDecoder | RowDecoderFactory | ColumnDecoder` to work with pulsar interface, and with some our own extensions compare to presto original version , we can support more type for backward compatible (e.g. ` TIMESTAMP\DATE\TIME\Real\ARRAY\MAP\ROW ` support). - Decouple avro or schema type with `pulsar-presto main module` (RecordSet,ConnectorMetadata .etc ), aim to friendly with other schema type ( [ProtobufNative](https://github.com/apache/pulsar/pull/8372) 、thrift etc..). ### Modifications Describe in [PIP-71: Pulsar SQL migrate SchemaHandle to presto decoder](https://docs.google.com/document/d/1KwG0GoHccju4-QNPfvT6tOwhp5Fvs6-iZlfLooPxTDM/edit?usp=sharing) ---- ### Does this pull request potentially affect one of the following parts: *If `yes` was chosen, please highlight the changes* - Dependencies (does it add or upgrade a dependency): (**yes** ) - The public API: (no) - The schema: ( no) - The default values of configurations: (no) - The wire protocol: (no) - The rest endpoints: (no) - The admin cli options: (no) - Anything that affects deployment: (no) ### Documentation - Does this pull request introduce a new feature? (yes) [[PIP][SQL]Migrate SchemaHandle to Presto-decoder](https://docs.google.com/document/d/1KwG0GoHccju4-QNPfvT6tOwhp5Fvs6-iZlfLooPxTDM/edit?usp=sharing) * codeStyle fix * Update pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/TestPulsarConnector.java Co-authored-by: ran * Update pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/TestPulsarConnector.java Co-authored-by: ran * Update pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/TestPulsarConnector.java Co-authored-by: ran * add keyValue\Primitive schema test && add schema cyclic definition detect * merge master * merge master Co-authored-by: wangguowei Co-authored-by: ran --- pulsar-sql/presto-distribution/LICENSE | 3 + pulsar-sql/presto-pulsar/pom.xml | 26 + .../pulsar/sql/presto/AvroSchemaHandler.java | 101 ---- .../pulsar/sql/presto/JSONSchemaHandler.java | 108 ---- .../sql/presto/KeyValueSchemaHandler.java | 105 ---- .../pulsar/sql/presto/PulsarColumnHandle.java | 75 ++- .../sql/presto/PulsarColumnMetadata.java | 128 ++++- .../sql/presto/PulsarConnectorModule.java | 8 + .../PulsarDispatchingRowDecoderFactory.java | 84 ++++ .../sql/presto/PulsarFieldValueProviders.java | 62 +++ .../sql/presto/PulsarInternalColumn.java | 173 +------ .../pulsar/sql/presto/PulsarMetadata.java | 279 +---------- .../presto/PulsarPrimitiveSchemaHandler.java | 61 --- .../pulsar/sql/presto/PulsarRecordCursor.java | 303 ++++++----- .../pulsar/sql/presto/PulsarRecordSet.java | 10 +- .../sql/presto/PulsarRecordSetProvider.java | 8 +- ...hemaHandler.java => PulsarRowDecoder.java} | 32 +- .../sql/presto/PulsarRowDecoderFactory.java | 51 ++ .../sql/presto/PulsarSchemaHandlers.java | 67 --- .../pulsar/sql/presto/PulsarSplitManager.java | 2 +- .../presto/PulsarSqlSchemaInfoProvider.java | 13 +- .../decoder/avro/PulsarAvroColumnDecoder.java | 385 ++++++++++++++ .../decoder/avro/PulsarAvroRowDecoder.java | 77 +++ .../avro/PulsarAvroRowDecoderFactory.java | 190 +++++++ .../sql/presto/decoder/avro/package-info.java | 22 + .../decoder/json/PulsarJsonFieldDecoder.java | 438 ++++++++++++++++ .../decoder/json/PulsarJsonRowDecoder.java | 89 ++++ .../json/PulsarJsonRowDecoderFactory.java | 186 +++++++ .../sql/presto/decoder/json/package-info.java | 22 + .../primitive/PulsarPrimitiveRowDecoder.java | 101 ++++ .../PulsarPrimitiveRowDecoderFactory.java | 115 +++++ .../decoder/primitive/package-info.java | 22 + .../sql/presto/TestAvroSchemaHandler.java | 127 ----- .../sql/presto/TestPulsarConnector.java | 471 ++++-------------- .../sql/presto/TestPulsarConnectorConfig.java | 2 +- .../TestPulsarKeyValueSchemaHandler.java | 353 ------------- .../pulsar/sql/presto/TestPulsarMetadata.java | 56 +-- .../TestPulsarPrimitiveSchemaHandler.java | 164 ------ .../sql/presto/TestPulsarRecordCursor.java | 391 +++++++++++---- .../presto/decoder/AbstractDecoderTester.java | 133 +++++ .../presto/decoder/DecoderTestMessage.java | 91 ++++ .../sql/presto/decoder/DecoderTestUtil.java | 121 +++++ .../decoder/avro/AvroDecoderTestUtil.java | 199 ++++++++ .../presto/decoder/avro/TestAvroDecoder.java | 290 +++++++++++ .../decoder/json/JsonDecoderTestUtil.java | 199 ++++++++ .../presto/decoder/json/TestJsonDecoder.java | 284 +++++++++++ .../primitive/PrimitiveDecoderTestUtil.java | 56 +++ .../primitive/TestPrimitiveDecoder.java | 233 +++++++++ 48 files changed, 4324 insertions(+), 2192 deletions(-) delete mode 100644 pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/AvroSchemaHandler.java delete mode 100644 pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/JSONSchemaHandler.java delete mode 100644 pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/KeyValueSchemaHandler.java create mode 100644 pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarDispatchingRowDecoderFactory.java create mode 100644 pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarFieldValueProviders.java delete mode 100644 pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarPrimitiveSchemaHandler.java rename pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/{SchemaHandler.java => PulsarRowDecoder.java} (54%) create mode 100644 pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarRowDecoderFactory.java delete mode 100644 pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarSchemaHandlers.java create mode 100644 pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/decoder/avro/PulsarAvroColumnDecoder.java create mode 100644 pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/decoder/avro/PulsarAvroRowDecoder.java create mode 100644 pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/decoder/avro/PulsarAvroRowDecoderFactory.java create mode 100644 pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/decoder/avro/package-info.java create mode 100644 pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/decoder/json/PulsarJsonFieldDecoder.java create mode 100644 pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/decoder/json/PulsarJsonRowDecoder.java create mode 100644 pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/decoder/json/PulsarJsonRowDecoderFactory.java create mode 100644 pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/decoder/json/package-info.java create mode 100644 pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/decoder/primitive/PulsarPrimitiveRowDecoder.java create mode 100644 pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/decoder/primitive/PulsarPrimitiveRowDecoderFactory.java create mode 100644 pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/decoder/primitive/package-info.java delete mode 100644 pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/TestAvroSchemaHandler.java delete mode 100644 pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/TestPulsarKeyValueSchemaHandler.java delete mode 100644 pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/TestPulsarPrimitiveSchemaHandler.java create mode 100644 pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/decoder/AbstractDecoderTester.java create mode 100644 pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/decoder/DecoderTestMessage.java create mode 100644 pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/decoder/DecoderTestUtil.java create mode 100644 pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/decoder/avro/AvroDecoderTestUtil.java create mode 100644 pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/decoder/avro/TestAvroDecoder.java create mode 100644 pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/decoder/json/JsonDecoderTestUtil.java create mode 100644 pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/decoder/json/TestJsonDecoder.java create mode 100644 pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/decoder/primitive/PrimitiveDecoderTestUtil.java create mode 100644 pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/decoder/primitive/TestPrimitiveDecoder.java diff --git a/pulsar-sql/presto-distribution/LICENSE b/pulsar-sql/presto-distribution/LICENSE index 4126581..60c3875 100644 --- a/pulsar-sql/presto-distribution/LICENSE +++ b/pulsar-sql/presto-distribution/LICENSE @@ -387,6 +387,7 @@ The Apache Software License, Version 2.0 - presto-parser-332.jar - presto-plugin-toolkit-332.jar - presto-spi-332.jar + - presto-record-decoder-332.jar * RocksDB JNI - rocksdbjni-6.10.2.jar * SnakeYAML @@ -433,6 +434,8 @@ The Apache Software License, Version 2.0 - commons-logging-1.2.jar * GSON - gson-2.8.6.jar + * Snappy + - snappy-java-1.1.7.3.jar * Jackson - jackson-module-parameter-names-2.10.0.jar - jackson-module-parameter-names-2.11.1.jar diff --git a/pulsar-sql/presto-pulsar/pom.xml b/pulsar-sql/presto-pulsar/pom.xml index b15e8e3..f99ce7f 100644 --- a/pulsar-sql/presto-pulsar/pom.xml +++ b/pulsar-sql/presto-pulsar/pom.xml @@ -101,6 +101,32 @@ ${joda.version} + + io.prestosql + presto-record-decoder + ${presto.version} + + + + ${project.groupId} + pulsar-client-original + ${project.version} + + + + io.prestosql + presto-main + ${presto.version} + test + + + + io.prestosql + presto-testing + ${presto.version} + test + + diff --git a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/AvroSchemaHandler.java b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/AvroSchemaHandler.java deleted file mode 100644 index db369cf..0000000 --- a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/AvroSchemaHandler.java +++ /dev/null @@ -1,101 +0,0 @@ -/** - * 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 com.google.common.annotations.VisibleForTesting; -import io.airlift.log.Logger; -import io.netty.buffer.ByteBuf; -import java.util.List; -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.naming.TopicName; -import org.apache.pulsar.common.schema.SchemaInfo; - - -/** - * Schema handler for payload in the Avro format. - */ -public class AvroSchemaHandler implements SchemaHandler { - - private final List columnHandles; - - private final GenericAvroSchema genericAvroSchema; - - private final SchemaInfo schemaInfo; - - private static final Logger log = Logger.get(AvroSchemaHandler.class); - - AvroSchemaHandler(TopicName topicName, - PulsarConnectorConfig pulsarConnectorConfig, - SchemaInfo schemaInfo, - List columnHandles) throws PulsarClientException { - this(new PulsarSqlSchemaInfoProvider(topicName, - pulsarConnectorConfig.getPulsarAdmin()), schemaInfo, columnHandles); - } - - AvroSchemaHandler(PulsarSqlSchemaInfoProvider pulsarSqlSchemaInfoProvider, - SchemaInfo schemaInfo, List columnHandles) { - this.schemaInfo = schemaInfo; - this.genericAvroSchema = new GenericAvroSchema(schemaInfo); - this.genericAvroSchema.setSchemaInfoProvider(pulsarSqlSchemaInfoProvider); - this.columnHandles = columnHandles; - } - - @Override - public Object deserialize(ByteBuf payload) { - return genericAvroSchema.decode(payload); - } - - @Override - public Object deserialize(ByteBuf payload, byte[] schemaVersion) { - return genericAvroSchema.decode(payload, schemaVersion); - } - - @Override - public Object extractField(int index, Object currentRecord) { - try { - GenericAvroRecord record = (GenericAvroRecord) currentRecord; - PulsarColumnHandle pulsarColumnHandle = this.columnHandles.get(index); - String[] names = pulsarColumnHandle.getFieldNames(); - - if (names.length == 1) { - return record.getField(pulsarColumnHandle.getFieldNames()[0]); - } else { - for (int i = 0; i < names.length - 1; i++) { - record = (GenericAvroRecord) record.getField(names[i]); - } - return record.getField(names[names.length - 1]); - } - } catch (Exception ex) { - log.debug(ex, "%s", ex); - } - return null; - } - - @VisibleForTesting - GenericAvroSchema getSchema() { - return this.genericAvroSchema; - } - - @VisibleForTesting - SchemaInfo getSchemaInfo() { - return schemaInfo; - } -} diff --git a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/JSONSchemaHandler.java b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/JSONSchemaHandler.java deleted file mode 100644 index a31aa41..0000000 --- a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/JSONSchemaHandler.java +++ /dev/null @@ -1,108 +0,0 @@ -/** - * 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 com.dslplatform.json.DslJson; -import io.airlift.log.Logger; -import io.netty.buffer.ByteBuf; -import io.netty.util.concurrent.FastThreadLocal; -import io.prestosql.spi.type.Type; -import java.io.IOException; -import java.math.BigDecimal; -import java.util.List; -import java.util.Map; - -/** - * Schema handler for payload in the JSON format. - */ -public class JSONSchemaHandler implements SchemaHandler { - - private static final Logger log = Logger.get(JSONSchemaHandler.class); - - private List columnHandles; - - private final DslJson dslJson = new DslJson<>(); - - private static final FastThreadLocal tmpBuffer = new FastThreadLocal() { - @Override - protected byte[] initialValue() { - return new byte[1024]; - } - }; - - public JSONSchemaHandler(List columnHandles) { - this.columnHandles = columnHandles; - } - - @Override - public Object deserialize(ByteBuf payload) { - // Since JSON deserializer only works on a byte[] we need to convert a direct mem buffer into - // a byte[]. - 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); - - try { - return dslJson.deserialize(Map.class, buffer, size); - } catch (IOException e) { - log.error("Failed to deserialize Json object", e); - return null; - } - } - - @Override - public Object extractField(int index, Object currentRecord) { - try { - Map jsonObject = (Map) currentRecord; - PulsarColumnHandle pulsarColumnHandle = columnHandles.get(index); - - String[] fieldNames = pulsarColumnHandle.getFieldNames(); - Object field = jsonObject.get(fieldNames[0]); - if (field == null) { - return null; - } - for (int i = 1; i < fieldNames.length; i++) { - field = ((Map) field).get(fieldNames[i]); - if (field == null) { - return null; - } - } - - Type type = pulsarColumnHandle.getType(); - - Class javaType = type.getJavaType(); - - if (javaType == double.class) { - return ((BigDecimal) field).doubleValue(); - } - - return field; - } catch (Exception ex) { - log.debug(ex, "%s", ex); - } - return null; - } -} diff --git a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/KeyValueSchemaHandler.java b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/KeyValueSchemaHandler.java deleted file mode 100644 index 434dd44..0000000 --- a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/KeyValueSchemaHandler.java +++ /dev/null @@ -1,105 +0,0 @@ -/** - * 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 com.google.common.annotations.VisibleForTesting; -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.naming.TopicName; -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 columnHandles; - - private final SchemaHandler keySchemaHandler; - - private final SchemaHandler valueSchemaHandler; - - private KeyValueEncodingType keyValueEncodingType; - - public KeyValueSchemaHandler(TopicName topicName, - PulsarConnectorConfig pulsarConnectorConfig, - SchemaInfo schemaInfo, - List columnHandles) { - this.columnHandles = columnHandles; - KeyValue kvSchemaInfo = KeyValueSchemaInfo.decodeKeyValueSchemaInfo(schemaInfo); - keySchemaHandler = PulsarSchemaHandlers.newPulsarSchemaHandler(topicName, pulsarConnectorConfig, - kvSchemaInfo.getKey(), columnHandles); - valueSchemaHandler = PulsarSchemaHandlers.newPulsarSchemaHandler(topicName, pulsarConnectorConfig, - kvSchemaInfo.getValue(), columnHandles); - keyValueEncodingType = KeyValueSchemaInfo.decodeKeyValueEncodingType(schemaInfo); - } - - @VisibleForTesting - KeyValueSchemaHandler(SchemaHandler keySchemaHandler, - SchemaHandler valueSchemaHandler, - List columnHandles) { - this.keySchemaHandler = keySchemaHandler; - this.valueSchemaHandler = valueSchemaHandler; - this.columnHandles = columnHandles; - } - - @Override - public Object deserialize(ByteBuf keyPayload, ByteBuf dataPayload, byte[] schemaVersion) { - Object keyObj; - Object valueObj; - 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; - } - - keyObj = keySchemaHandler.deserialize(keyByteBuf, schemaVersion); - valueObj = valueSchemaHandler.deserialize(valueByteBuf, schemaVersion); - return new KeyValue<>(keyObj, valueObj); - } - - @Override - public Object extractField(int index, Object currentRecord) { - PulsarColumnHandle pulsarColumnHandle = this.columnHandles.get(index); - KeyValue keyValue = (KeyValue) currentRecord; - if (pulsarColumnHandle.isKey()) { - return keySchemaHandler.extractField(index, keyValue.getKey()); - } else if (pulsarColumnHandle.isValue()) { - return valueSchemaHandler.extractField(index, keyValue.getValue()); - } - return null; - } - -} diff --git a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarColumnHandle.java b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarColumnHandle.java index a8763d5..6070247 100644 --- a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarColumnHandle.java +++ b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarColumnHandle.java @@ -22,16 +22,16 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonProperty; -import io.prestosql.spi.connector.ColumnHandle; +import io.prestosql.decoder.DecoderColumnHandle; import io.prestosql.spi.connector.ColumnMetadata; import io.prestosql.spi.type.Type; -import java.util.Arrays; + import java.util.Objects; /** * This class represents the basic information about a presto column. */ -public class PulsarColumnHandle implements ColumnHandle { +public class PulsarColumnHandle implements DecoderColumnHandle { private final String connectorId; @@ -55,12 +55,23 @@ public class PulsarColumnHandle implements ColumnHandle { */ private final boolean internal; - private final String[] fieldNames; - - private final Integer[] positionIndices; private HandleKeyValueType handleKeyValueType; + /** + * {@link org.apache.pulsar.sql.presto.PulsarColumnMetadata.DecoderExtraInfo#mapping}. + */ + private String mapping; + /** + * {@link org.apache.pulsar.sql.presto.PulsarColumnMetadata.DecoderExtraInfo#dataFormat}. + */ + private String dataFormat; + + /** + * {@link org.apache.pulsar.sql.presto.PulsarColumnMetadata.DecoderExtraInfo#formatHint}. + */ + private String formatHint; + /** * Column Handle keyValue type, used for keyValue schema. */ @@ -86,16 +97,18 @@ public PulsarColumnHandle( @JsonProperty("type") Type type, @JsonProperty("hidden") boolean hidden, @JsonProperty("internal") boolean internal, - @JsonProperty("fieldNames") String[] fieldNames, - @JsonProperty("positionIndices") Integer[] positionIndices, + @JsonProperty("mapping") String mapping, + @JsonProperty("dataFormat") String dataFormat, + @JsonProperty("formatHint") String formatHint, @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; + this.mapping = mapping; + this.dataFormat = dataFormat; + this.formatHint = formatHint; if (handleKeyValueType == null) { this.handleKeyValueType = HandleKeyValueType.NONE; } else { @@ -113,6 +126,16 @@ public String getName() { return name; } + @JsonProperty + public String getMapping() { + return mapping; + } + + @JsonProperty + public String getDataFormat() { + return dataFormat; + } + @JsonProperty public Type getType() { return type; @@ -129,13 +152,8 @@ public boolean isInternal() { } @JsonProperty - public String[] getFieldNames() { - return fieldNames; - } - - @JsonProperty - public Integer[] getPositionIndices() { - return positionIndices; + public String getFormatHint() { + return formatHint; } @JsonProperty @@ -154,7 +172,9 @@ public boolean isValue() { } ColumnMetadata getColumnMetadata() { - return new ColumnMetadata(name, type, null, hidden); + return new PulsarColumnMetadata(name, type, null, null, hidden, + internal, handleKeyValueType, new PulsarColumnMetadata.DecoderExtraInfo( + mapping, dataFormat, formatHint)); } @Override @@ -183,12 +203,17 @@ public boolean equals(Object o) { if (type != null ? !type.equals(that.type) : that.type != null) { return false; } - if (!Arrays.deepEquals(fieldNames, that.fieldNames)) { + if (mapping != null ? !mapping.equals(that.mapping) : that.mapping != null) { return false; } - if (!Arrays.deepEquals(positionIndices, that.positionIndices)) { + if (dataFormat != null ? !dataFormat.equals(that.dataFormat) : that.dataFormat != null) { return false; } + + if (formatHint != null ? !formatHint.equals(that.formatHint) : that.formatHint != null) { + return false; + } + return Objects.equals(handleKeyValueType, that.handleKeyValueType); } @@ -199,8 +224,9 @@ public int hashCode() { result = 31 * result + (type != null ? type.hashCode() : 0); result = 31 * result + (hidden ? 1 : 0); result = 31 * result + (internal ? 1 : 0); - result = 31 * result + Arrays.hashCode(fieldNames); - result = 31 * result + Arrays.hashCode(positionIndices); + result = 31 * result + (mapping != null ? mapping.hashCode() : 0); + result = 31 * result + (dataFormat != null ? dataFormat.hashCode() : 0); + result = 31 * result + (formatHint != null ? formatHint.hashCode() : 0); result = 31 * result + (handleKeyValueType != null ? handleKeyValueType.hashCode() : 0); return result; } @@ -213,8 +239,9 @@ public String toString() { + ", type=" + type + ", hidden=" + hidden + ", internal=" + internal - + ", fieldNames=" + Arrays.toString(fieldNames) - + ", positionIndices=" + Arrays.toString(positionIndices) + + ", mapping=" + mapping + + ", dataFormat=" + dataFormat + + ", formatHint=" + formatHint + ", handleKeyValueType=" + handleKeyValueType + '}'; } diff --git a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarColumnMetadata.java b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarColumnMetadata.java index a24367d..4ba153d 100644 --- a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarColumnMetadata.java +++ b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarColumnMetadata.java @@ -20,7 +20,7 @@ import io.prestosql.spi.connector.ColumnMetadata; import io.prestosql.spi.type.Type; -import java.util.Arrays; + import java.util.Objects; /** @@ -31,23 +31,27 @@ public class PulsarColumnMetadata extends ColumnMetadata { private boolean isInternal; // need this because presto ColumnMetadata saves name in lowercase private String nameWithCase; - private String[] fieldNames; - private Integer[] positionIndices; private PulsarColumnHandle.HandleKeyValueType handleKeyValueType; public final static String KEY_SCHEMA_COLUMN_PREFIX = "__key."; + private DecoderExtraInfo decoderExtraInfo; + public PulsarColumnMetadata(String name, Type type, String comment, String extraInfo, boolean hidden, boolean isInternal, - String[] fieldNames, Integer[] positionIndices, - PulsarColumnHandle.HandleKeyValueType handleKeyValueType) { + PulsarColumnHandle.HandleKeyValueType handleKeyValueType, + DecoderExtraInfo decoderExtraInfo) { super(name, type, comment, extraInfo, hidden); this.nameWithCase = name; this.isInternal = isInternal; - this.fieldNames = fieldNames; - this.positionIndices = positionIndices; this.handleKeyValueType = handleKeyValueType; + this.decoderExtraInfo = decoderExtraInfo; } + public DecoderExtraInfo getDecoderExtraInfo() { + return decoderExtraInfo; + } + + public String getNameWithCase() { return nameWithCase; } @@ -56,13 +60,6 @@ public boolean isInternal() { return isInternal; } - public String[] getFieldNames() { - return fieldNames; - } - - public Integer[] getPositionIndices() { - return positionIndices; - } public PulsarColumnHandle.HandleKeyValueType getHandleKeyValueType() { return handleKeyValueType; @@ -88,9 +85,8 @@ public String toString() { return "PulsarColumnMetadata{" + "isInternal=" + isInternal + ", nameWithCase='" + nameWithCase + '\'' - + ", fieldNames=" + Arrays.toString(fieldNames) - + ", positionIndices=" + Arrays.toString(positionIndices) + ", handleKeyValueType=" + handleKeyValueType + + ", decoderExtraInfo=" + decoderExtraInfo.toString() + '}'; } @@ -114,10 +110,7 @@ public boolean equals(Object o) { if (nameWithCase != null ? !nameWithCase.equals(that.nameWithCase) : that.nameWithCase != null) { return false; } - if (!Arrays.deepEquals(fieldNames, that.fieldNames)) { - return false; - } - if (!Arrays.deepEquals(positionIndices, that.positionIndices)) { + if (decoderExtraInfo != null ? !decoderExtraInfo.equals(that.decoderExtraInfo) : that.decoderExtraInfo != null) { return false; } return Objects.equals(handleKeyValueType, that.handleKeyValueType); @@ -128,9 +121,100 @@ public int hashCode() { int result = super.hashCode(); result = 31 * result + (isInternal ? 1 : 0); result = 31 * result + (nameWithCase != null ? nameWithCase.hashCode() : 0); - result = 31 * result + Arrays.hashCode(fieldNames); - result = 31 * result + Arrays.hashCode(positionIndices); + result = 31 * result + (decoderExtraInfo != null ? decoderExtraInfo.hashCode() : 0); result = 31 * result + (handleKeyValueType != null ? handleKeyValueType.hashCode() : 0); return result; } + + + /** + * Decoder extra info for {@link org.apache.pulsar.sql.presto.PulsarColumnHandle} + * used by {@link io.prestosql.decoder.RowDecoder}. + */ + public static class DecoderExtraInfo { + + public DecoderExtraInfo(String mapping, String dataFormat, String formatHint) { + this.mapping = mapping; + this.dataFormat = dataFormat; + this.formatHint = formatHint; + } + + public DecoderExtraInfo() {} + + //equals ColumnName in general, may used as alias or embedded field in future. + private String mapping; + //reserved dataFormat used by RowDecoder. + private String dataFormat; + //reserved formatHint used by RowDecoder. + private String formatHint; + + public String getMapping() { + return mapping; + } + + public void setMapping(String mapping) { + this.mapping = mapping; + } + + public String getDataFormat() { + return dataFormat; + } + + public void setDataFormat(String dataFormat) { + this.dataFormat = dataFormat; + } + + public String getFormatHint() { + return formatHint; + } + + public void setFormatHint(String formatHint) { + this.formatHint = formatHint; + } + + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + if (!super.equals(o)) { + return false; + } + + DecoderExtraInfo that = (DecoderExtraInfo) o; + + if (mapping != that.mapping) { + return false; + } + if (dataFormat != null ? !dataFormat.equals(that.dataFormat) : that.dataFormat != null) { + return false; + } + return Objects.equals(formatHint, that.formatHint); + } + + @Override + public String toString() { + return "DecoderExtraInfo{" + + "mapping=" + mapping + + ", dataFormat=" + dataFormat + + ", formatHint=" + formatHint + + '}'; + } + + @Override + public int hashCode() { + int result = super.hashCode(); + result = 31 * result + (mapping != null ? mapping.hashCode() : 0); + result = 31 * result + (dataFormat != null ? dataFormat.hashCode() : 0); + result = 31 * result + (formatHint != null ? formatHint.hashCode() : 0); + return result; + } + + } + + } diff --git a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarConnectorModule.java b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarConnectorModule.java index 7e2d274..881d015 100644 --- a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarConnectorModule.java +++ b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarConnectorModule.java @@ -26,9 +26,11 @@ import com.google.inject.Binder; import com.google.inject.Module; import com.google.inject.Scopes; +import io.prestosql.decoder.DecoderModule; import io.prestosql.spi.type.Type; import io.prestosql.spi.type.TypeId; import io.prestosql.spi.type.TypeManager; + import javax.inject.Inject; /** @@ -54,9 +56,15 @@ public void configure(Binder binder) { binder.bind(PulsarMetadata.class).in(Scopes.SINGLETON); binder.bind(PulsarSplitManager.class).in(Scopes.SINGLETON); binder.bind(PulsarRecordSetProvider.class).in(Scopes.SINGLETON); + + binder.bind(PulsarDispatchingRowDecoderFactory.class).in(Scopes.SINGLETON); + configBinder(binder).bindConfig(PulsarConnectorConfig.class); jsonBinder(binder).addDeserializerBinding(Type.class).to(TypeDeserializer.class); + + binder.install(new DecoderModule()); + } /** diff --git a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarDispatchingRowDecoderFactory.java b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarDispatchingRowDecoderFactory.java new file mode 100644 index 0000000..9fc6f99 --- /dev/null +++ b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarDispatchingRowDecoderFactory.java @@ -0,0 +1,84 @@ +/** + * 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 static java.lang.String.format; + +import com.google.inject.Inject; + +import io.airlift.log.Logger; + +import io.prestosql.decoder.DecoderColumnHandle; +import io.prestosql.spi.connector.ColumnMetadata; +import io.prestosql.spi.type.TypeManager; + +import java.util.List; +import java.util.Set; + +import org.apache.pulsar.common.naming.TopicName; +import org.apache.pulsar.common.schema.SchemaInfo; +import org.apache.pulsar.common.schema.SchemaType; +import org.apache.pulsar.sql.presto.decoder.avro.PulsarAvroRowDecoderFactory; +import org.apache.pulsar.sql.presto.decoder.json.PulsarJsonRowDecoderFactory; +import org.apache.pulsar.sql.presto.decoder.primitive.PulsarPrimitiveRowDecoderFactory; + +/** + * dispatcher RowDecoderFactory for {@link org.apache.pulsar.common.schema.SchemaType}. + */ +public class PulsarDispatchingRowDecoderFactory { + + private static final Logger log = Logger.get(PulsarDispatchingRowDecoderFactory.class); + + private TypeManager typeManager; + + @Inject + public PulsarDispatchingRowDecoderFactory(TypeManager typeManager) { + this.typeManager = typeManager; + } + + public PulsarRowDecoder createRowDecoder(TopicName topicName, SchemaInfo schemaInfo, + Set columns) { + PulsarRowDecoderFactory rowDecoderFactory = createDecoderFactory(schemaInfo); + return rowDecoderFactory.createRowDecoder(topicName, schemaInfo, columns); + } + + public List extractColumnMetadata(TopicName topicName, SchemaInfo schemaInfo, + PulsarColumnHandle.HandleKeyValueType handleKeyValueType) { + PulsarRowDecoderFactory rowDecoderFactory = createDecoderFactory(schemaInfo); + return rowDecoderFactory.extractColumnMetadata(topicName, schemaInfo, handleKeyValueType); + } + + private PulsarRowDecoderFactory createDecoderFactory(SchemaInfo schemaInfo) { + if (SchemaType.AVRO.equals(schemaInfo.getType())) { + return new PulsarAvroRowDecoderFactory(typeManager); + } else if (SchemaType.JSON.equals(schemaInfo.getType())) { + return new PulsarJsonRowDecoderFactory(typeManager); + } else if (schemaInfo.getType().isPrimitive()) { + return new PulsarPrimitiveRowDecoderFactory(); + } else { + throw new RuntimeException(format("'%s' is unsupported type '%s'", schemaInfo.getName(), + schemaInfo.getType())); + } + } + + public TypeManager getTypeManager() { + return typeManager; + } + +} \ No newline at end of file diff --git a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarFieldValueProviders.java b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarFieldValueProviders.java new file mode 100644 index 0000000..c1ef3c8 --- /dev/null +++ b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarFieldValueProviders.java @@ -0,0 +1,62 @@ +/** + * 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.prestosql.decoder.FieldValueProvider; + +/** + * custom FieldValueProvider for Pulsar. + */ +public class PulsarFieldValueProviders { + + public static FieldValueProvider doubleValueProvider(double value) { + return new FieldValueProvider() { + @Override + public double getDouble() { + return value; + } + + @Override + public boolean isNull() { + return false; + } + }; + } + + /** + * FieldValueProvider for Time (Data,Timstamp etc.) with indicate Null instead of longValueProvider. + * @param value + * @param isNull + * @return + */ + public static FieldValueProvider timeValueProvider(long value, boolean isNull) { + return new FieldValueProvider() { + @Override + public long getLong() { + return value; + } + + @Override + public boolean isNull() { + return isNull; + } + }; + } + +} diff --git a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarInternalColumn.java b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarInternalColumn.java index 763f2e2..9340977 100644 --- a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarInternalColumn.java +++ b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarInternalColumn.java @@ -21,8 +21,7 @@ import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Strings.isNullOrEmpty; import static java.util.Objects.requireNonNull; -import com.fasterxml.jackson.core.JsonProcessingException; -import com.fasterxml.jackson.databind.ObjectMapper; + import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; import io.prestosql.spi.type.BigintType; @@ -33,162 +32,40 @@ import java.util.Map; import java.util.Set; import java.util.function.Consumer; -import org.apache.pulsar.common.api.raw.RawMessage; /** * This abstract class represents internal columns. */ -public abstract class PulsarInternalColumn { - - /** - * Internal column representing the partition. - */ - public static class PartitionColumn extends PulsarInternalColumn { - - PartitionColumn(String name, Type type, String comment) { - super(name, type, comment); - } - - @Override - public Object getData(RawMessage message) { - return null; - } - } - - /** - * Internal column representing the event time. - */ - public static class EventTimeColumn extends PulsarInternalColumn { - - EventTimeColumn(String name, Type type, String comment) { - super(name, type, comment); - } - - @Override - public Object getData(RawMessage message) { - return message.getEventTime() == 0 ? null : message.getEventTime(); - } - } - - /** - * Internal column representing the publish time. - */ - public static class PublishTimeColumn extends PulsarInternalColumn { - - PublishTimeColumn(String name, Type type, String comment) { - super(name, type, comment); - } - - @Override - public Object getData(RawMessage message) { - return message.getPublishTime(); - } - } - - /** - * Internal column representing the message id. - */ - public static class MessageIdColumn extends PulsarInternalColumn { - - MessageIdColumn(String name, Type type, String comment) { - super(name, type, comment); - } - - @Override - public Object getData(RawMessage message) { - return message.getMessageId().toString(); - } - } +public class PulsarInternalColumn { - /** - * Internal column representing the sequence id. - */ - public static class SequenceIdColumn extends PulsarInternalColumn { - SequenceIdColumn(String name, Type type, String comment) { - super(name, type, comment); - } + public static final PulsarInternalColumn PARTITION = new PulsarInternalColumn("__partition__", + IntegerType.INTEGER, "The partition number which the message belongs to"); - @Override - public Object getData(RawMessage message) { - return message.getSequenceId(); - } - } - - /** - * Internal column representing the producer name. - */ - public static class ProducerNameColumn extends PulsarInternalColumn { - - ProducerNameColumn(String name, Type type, String comment) { - super(name, type, comment); - } - - @Override - public Object getData(RawMessage message) { - return message.getProducerName(); - } - } - - /** - * Internal column representing the key. - */ - public static class KeyColumn extends PulsarInternalColumn { - - KeyColumn(String name, Type type, String comment) { - super(name, type, comment); - } - - @Override - public Object getData(RawMessage message) { - return message.getKey().orElse(null); - } - } + public static final PulsarInternalColumn EVENT_TIME = new PulsarInternalColumn("__event_time__", + TimestampType.TIMESTAMP, "Application defined timestamp in milliseconds of when the event occurred"); - /** - * Internal column representing the message properties. - */ - public static class PropertiesColumn extends PulsarInternalColumn { - - private static final ObjectMapper mapper = new ObjectMapper(); - - PropertiesColumn(String name, Type type, String comment) { - super(name, type, comment); - } - - @Override - public Object getData(RawMessage message) { - try { - return mapper.writeValueAsString(message.getProperties()); - } catch (JsonProcessingException e) { - throw new RuntimeException(e); - } - } - } - - public static final PartitionColumn PARTITION = new PartitionColumn("__partition__", IntegerType.INTEGER, - "The partition number which the message belongs to"); + public static final PulsarInternalColumn PUBLISH_TIME = new PulsarInternalColumn("__publish_time__", + TimestampType.TIMESTAMP, "The timestamp in milliseconds of when event as published"); - public static final PulsarInternalColumn EVENT_TIME = new EventTimeColumn("__event_time__", TimestampType - .TIMESTAMP, "Application defined timestamp in milliseconds of when the event occurred"); + public static final PulsarInternalColumn MESSAGE_ID = new PulsarInternalColumn("__message_id__", + VarcharType.VARCHAR, "The message ID of the message used to generate this row"); - public static final PulsarInternalColumn PUBLISH_TIME = new PublishTimeColumn("__publish_time__", - TimestampType.TIMESTAMP, "The timestamp in milliseconds of when event as published"); + public static final PulsarInternalColumn SEQUENCE_ID = new PulsarInternalColumn("__sequence_id__", + BigintType.BIGINT, "The sequence ID of the message used to generate this row"); - public static final PulsarInternalColumn MESSAGE_ID = new MessageIdColumn("__message_id__", VarcharType.VARCHAR, - "The message ID of the message used to generate this row"); + public static final PulsarInternalColumn PRODUCER_NAME = new PulsarInternalColumn("__producer_name__", + VarcharType.VARCHAR, "The name of the producer that publish the message used to generate this row"); - public static final PulsarInternalColumn SEQUENCE_ID = new SequenceIdColumn("__sequence_id__", BigintType.BIGINT, - "The sequence ID of the message used to generate this row"); + public static final PulsarInternalColumn KEY = new PulsarInternalColumn("__key__", + VarcharType.VARCHAR, "The partition key for the topic"); - public static final PulsarInternalColumn PRODUCER_NAME = new ProducerNameColumn("__producer_name__", VarcharType - .VARCHAR, "The name of the producer that publish the message used to generate this row"); + public static final PulsarInternalColumn PROPERTIES = new PulsarInternalColumn("__properties__", + VarcharType.VARCHAR, "User defined properties"); - public static final PulsarInternalColumn KEY = new KeyColumn("__key__", VarcharType.VARCHAR, "The partition key " - + "for the topic"); + private static Set internalFields = ImmutableSet.of(PARTITION, EVENT_TIME, PUBLISH_TIME, + MESSAGE_ID, SEQUENCE_ID, PRODUCER_NAME, KEY, PROPERTIES); - public static final PulsarInternalColumn PROPERTIES = new PropertiesColumn("__properties__", VarcharType.VARCHAR, - "User defined properties"); private final String name; private final Type type; @@ -217,17 +94,16 @@ PulsarColumnHandle getColumnHandle(String connectorId, boolean hidden) { getName(), getType(), hidden, - true, null, null, PulsarColumnHandle.HandleKeyValueType.NONE); + true, getName(), null, null, PulsarColumnHandle.HandleKeyValueType.NONE); } PulsarColumnMetadata getColumnMetadata(boolean hidden) { - return new PulsarColumnMetadata(name, type, comment, null, hidden, true, null, null, - PulsarColumnHandle.HandleKeyValueType.NONE); + return new PulsarColumnMetadata(name, type, comment, null, hidden, true, + PulsarColumnHandle.HandleKeyValueType.NONE, new PulsarColumnMetadata.DecoderExtraInfo()); } public static Set getInternalFields() { - return ImmutableSet.of(PARTITION, EVENT_TIME, PUBLISH_TIME, MESSAGE_ID, SEQUENCE_ID, PRODUCER_NAME, KEY, - PROPERTIES); + return internalFields; } public static Map getInternalFieldsMap() { @@ -241,5 +117,4 @@ public void accept(PulsarInternalColumn pulsarInternalColumn) { return builder.build(); } - public abstract Object getData(RawMessage message); } diff --git a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarMetadata.java b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarMetadata.java index fe27c29..0a7dfb8 100644 --- a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarMetadata.java +++ b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarMetadata.java @@ -19,16 +19,13 @@ package org.apache.pulsar.sql.presto; import static io.prestosql.spi.StandardErrorCode.NOT_FOUND; -import static io.prestosql.spi.StandardErrorCode.NOT_SUPPORTED; import static io.prestosql.spi.StandardErrorCode.QUERY_REJECTED; -import static io.prestosql.spi.type.DateType.DATE; -import static io.prestosql.spi.type.TimeType.TIME; -import static io.prestosql.spi.type.TimestampType.TIMESTAMP; import static java.util.Objects.requireNonNull; import static org.apache.pulsar.sql.presto.PulsarConnectorUtils.restoreNamespaceDelimiterIfNeeded; import static org.apache.pulsar.sql.presto.PulsarConnectorUtils.rewriteNamespaceDelimiterIfNeeded; import static org.apache.pulsar.sql.presto.PulsarHandleResolver.convertColumnHandle; import static org.apache.pulsar.sql.presto.PulsarHandleResolver.convertTableHandle; + import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; @@ -47,34 +44,14 @@ import io.prestosql.spi.connector.SchemaTableName; import io.prestosql.spi.connector.SchemaTablePrefix; import io.prestosql.spi.connector.TableNotFoundException; -import io.prestosql.spi.type.BigintType; -import io.prestosql.spi.type.BooleanType; -import io.prestosql.spi.type.DateType; -import io.prestosql.spi.type.DoubleType; -import io.prestosql.spi.type.IntegerType; -import io.prestosql.spi.type.RealType; -import io.prestosql.spi.type.SmallintType; -import io.prestosql.spi.type.TimeType; -import io.prestosql.spi.type.TimestampType; -import io.prestosql.spi.type.TinyintType; -import io.prestosql.spi.type.Type; -import io.prestosql.spi.type.VarbinaryType; -import io.prestosql.spi.type.VarcharType; import java.util.HashMap; -import java.util.HashSet; import java.util.LinkedList; import java.util.List; import java.util.Map; import java.util.Optional; import java.util.Set; -import java.util.Stack; import java.util.stream.Collectors; import javax.inject.Inject; -import org.apache.avro.LogicalType; -import org.apache.avro.LogicalTypes; -import org.apache.avro.Schema; -import org.apache.avro.SchemaParseException; -import org.apache.commons.lang3.StringUtils; import org.apache.commons.lang3.exception.ExceptionUtils; import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.admin.PulsarAdminException; @@ -94,12 +71,17 @@ public class PulsarMetadata implements ConnectorMetadata { private final PulsarAdmin pulsarAdmin; private final PulsarConnectorConfig pulsarConnectorConfig; + private final PulsarDispatchingRowDecoderFactory decoderFactory; + private static final String INFORMATION_SCHEMA = "information_schema"; + private static final Logger log = Logger.get(PulsarMetadata.class); @Inject - public PulsarMetadata(PulsarConnectorId connectorId, PulsarConnectorConfig pulsarConnectorConfig) { + public PulsarMetadata(PulsarConnectorId connectorId, PulsarConnectorConfig pulsarConnectorConfig, + PulsarDispatchingRowDecoderFactory decoderFactory) { + this.decoderFactory = decoderFactory; this.connectorId = requireNonNull(connectorId, "connectorId is null").toString(); this.pulsarConnectorConfig = pulsarConnectorConfig; try { @@ -223,8 +205,9 @@ public Map getColumnHandles(ConnectorSession session, Conn pulsarColumnMetadata.getType(), pulsarColumnMetadata.isHidden(), pulsarColumnMetadata.isInternal(), - pulsarColumnMetadata.getFieldNames(), - pulsarColumnMetadata.getPositionIndices(), + pulsarColumnMetadata.getDecoderExtraInfo().getMapping(), + pulsarColumnMetadata.getDecoderExtraInfo().getDataFormat(), + pulsarColumnMetadata.getDecoderExtraInfo().getFormatHint(), pulsarColumnMetadata.getHandleKeyValueType()); columnHandles.put( @@ -315,7 +298,8 @@ private ConnectorTableMetadata getTableMetadata(SchemaTableName schemaTableName, } catch (PulsarAdminException e) { if (e.getStatusCode() == 404) { // use default schema because there is no schema - schemaInfo = PulsarSchemaHandlers.defaultSchema(); + schemaInfo = PulsarSqlSchemaInfoProvider.defaultSchema(); + } else if (e.getStatusCode() == 401) { throw new PrestoException(QUERY_REJECTED, String.format("Failed to get pulsar topic schema information for topic %s/%s: Unauthorized", @@ -337,15 +321,14 @@ private ConnectorTableMetadata getTableMetadata(SchemaTableName schemaTableName, /** * Convert pulsar schema into presto table metadata. */ - static List getPulsarColumns(TopicName topicName, + @VisibleForTesting + public List getPulsarColumns(TopicName topicName, SchemaInfo schemaInfo, boolean withInternalColumns, PulsarColumnHandle.HandleKeyValueType handleKeyValueType) { SchemaType schemaType = schemaInfo.getType(); - if (schemaType.isStruct()) { - return getPulsarColumnsFromStructSchema(topicName, schemaInfo, withInternalColumns, handleKeyValueType); - } else if (schemaType.isPrimitive()) { - return getPulsarColumnsFromPrimitiveSchema(topicName, schemaInfo, withInternalColumns, handleKeyValueType); + if (schemaType.isStruct() || schemaType.isPrimitive()) { + return getPulsarColumnsFromSchema(topicName, schemaInfo, withInternalColumns, handleKeyValueType); } else if (schemaType.equals(SchemaType.KEY_VALUE)) { return getPulsarColumnsFromKeyValueSchema(topicName, schemaInfo, withInternalColumns); } else { @@ -353,62 +336,23 @@ static List getPulsarColumns(TopicName topicName, } } - static List getPulsarColumnsFromPrimitiveSchema(TopicName topicName, - SchemaInfo schemaInfo, - boolean withInternalColumns, - PulsarColumnHandle.HandleKeyValueType handleKeyValueType) { + List getPulsarColumnsFromSchema(TopicName topicName, + SchemaInfo schemaInfo, + boolean withInternalColumns, + PulsarColumnHandle.HandleKeyValueType handleKeyValueType) { ImmutableList.Builder builder = ImmutableList.builder(); - - ColumnMetadata valueColumn = new PulsarColumnMetadata( - PulsarColumnMetadata.getColumnName(handleKeyValueType, "__value__"), - convertPulsarType(schemaInfo.getType()), - "The value of the message with primitive type schema", null, false, false, - new String[0], - new Integer[0], handleKeyValueType); - - builder.add(valueColumn); - + builder.addAll(decoderFactory.extractColumnMetadata(topicName, schemaInfo, handleKeyValueType)); if (withInternalColumns) { PulsarInternalColumn.getInternalFields() .stream() .forEach(pulsarInternalColumn -> builder.add(pulsarInternalColumn.getColumnMetadata(false))); } - return builder.build(); } - static List getPulsarColumnsFromStructSchema(TopicName topicName, - SchemaInfo schemaInfo, - boolean withInternalColumns, - PulsarColumnHandle.HandleKeyValueType handleKeyValueType) { - String schemaJson = new String(schemaInfo.getSchema()); - if (StringUtils.isBlank(schemaJson)) { - throw new PrestoException(NOT_SUPPORTED, "Topic " + topicName.toString() - + " does not have a valid schema"); - } - Schema schema; - try { - schema = PulsarConnectorUtils.parseSchema(schemaJson); - } catch (SchemaParseException ex) { - throw new PrestoException(NOT_SUPPORTED, "Topic " + topicName.toString() - + " does not have a valid schema"); - } - - ImmutableList.Builder builder = ImmutableList.builder(); - - builder.addAll(getColumns(null, schema, new HashSet<>(), new Stack<>(), new Stack<>(), handleKeyValueType)); - - if (withInternalColumns) { - PulsarInternalColumn.getInternalFields() - .stream() - .forEach(pulsarInternalColumn -> builder.add(pulsarInternalColumn.getColumnMetadata(false))); - } - return builder.build(); - } - - static List getPulsarColumnsFromKeyValueSchema(TopicName topicName, - SchemaInfo schemaInfo, - boolean withInternalColumns) { + List getPulsarColumnsFromKeyValueSchema(TopicName topicName, + SchemaInfo schemaInfo, + boolean withInternalColumns) { ImmutableList.Builder builder = ImmutableList.builder(); KeyValue kvSchemaInfo = KeyValueSchemaInfo.decodeKeyValueSchemaInfo(schemaInfo); SchemaInfo keySchemaInfo = kvSchemaInfo.getKey(); @@ -428,179 +372,4 @@ static List getPulsarColumnsFromKeyValueSchema(TopicName topicNa return builder.build(); } - @VisibleForTesting - static Type convertPulsarType(SchemaType pulsarType) { - switch (pulsarType) { - case BOOLEAN: - return BooleanType.BOOLEAN; - case INT8: - return TinyintType.TINYINT; - case INT16: - return SmallintType.SMALLINT; - case INT32: - return IntegerType.INTEGER; - case INT64: - return BigintType.BIGINT; - case FLOAT: - return RealType.REAL; - case DOUBLE: - return DoubleType.DOUBLE; - case NONE: - case BYTES: - return VarbinaryType.VARBINARY; - case STRING: - return VarcharType.VARCHAR; - case DATE: - return DateType.DATE; - case TIME: - return TimeType.TIME; - case TIMESTAMP: - return TimestampType.TIMESTAMP; - default: - log.error("Cannot convert type: %s", pulsarType); - return null; - } - } - - - @VisibleForTesting - static List getColumns(String fieldName, Schema fieldSchema, - Set fieldTypes, - Stack fieldNames, - Stack positionIndices, - PulsarColumnHandle.HandleKeyValueType handleKeyValueType) { - - List columnMetadataList = new LinkedList<>(); - - if (isPrimitiveType(fieldSchema.getType())) { - columnMetadataList.add(new PulsarColumnMetadata( - PulsarColumnMetadata.getColumnName(handleKeyValueType, fieldName), - convertType(fieldSchema.getType(), fieldSchema.getLogicalType()), - null, null, false, false, - fieldNames.toArray(new String[fieldNames.size()]), - positionIndices.toArray(new Integer[positionIndices.size()]), handleKeyValueType)); - } else if (fieldSchema.getType() == Schema.Type.UNION) { - boolean canBeNull = false; - for (Schema type : fieldSchema.getTypes()) { - if (isPrimitiveType(type.getType())) { - PulsarColumnMetadata columnMetadata; - if (type.getType() != Schema.Type.NULL) { - if (!canBeNull) { - columnMetadata = new PulsarColumnMetadata( - PulsarColumnMetadata.getColumnName(handleKeyValueType, fieldName), - convertType(type.getType(), type.getLogicalType()), - null, null, false, false, - fieldNames.toArray(new String[fieldNames.size()]), - positionIndices.toArray(new Integer[positionIndices.size()]), handleKeyValueType); - } else { - columnMetadata = new PulsarColumnMetadata( - PulsarColumnMetadata.getColumnName(handleKeyValueType, fieldName), - convertType(type.getType(), type.getLogicalType()), - "field can be null", null, false, false, - fieldNames.toArray(new String[fieldNames.size()]), - positionIndices.toArray(new Integer[positionIndices.size()]), handleKeyValueType); - } - columnMetadataList.add(columnMetadata); - } else { - canBeNull = true; - } - } else { - List columns = getColumns(fieldName, type, fieldTypes, fieldNames, - positionIndices, handleKeyValueType); - columnMetadataList.addAll(columns); - } - } - } else if (fieldSchema.getType() == Schema.Type.RECORD) { - // check if we have seen this type before to prevent cyclic class definitions. - if (!fieldTypes.contains(fieldSchema.getFullName())) { - // add to types seen so far in traversal - fieldTypes.add(fieldSchema.getFullName()); - List fields = fieldSchema.getFields(); - for (int i = 0; i < fields.size(); i++) { - Schema.Field field = fields.get(i); - fieldNames.push(field.name()); - positionIndices.push(i); - List columns; - if (fieldName == null) { - columns = getColumns(field.name(), field.schema(), fieldTypes, fieldNames, positionIndices, - handleKeyValueType); - } else { - columns = getColumns(String.format("%s.%s", fieldName, field.name()), field.schema(), - fieldTypes, fieldNames, positionIndices, handleKeyValueType); - - } - positionIndices.pop(); - fieldNames.pop(); - columnMetadataList.addAll(columns); - } - fieldTypes.remove(fieldSchema.getFullName()); - } else { - log.debug("Already seen type: %s", fieldSchema.getFullName()); - } - } else if (fieldSchema.getType() == Schema.Type.ARRAY) { - - } else if (fieldSchema.getType() == Schema.Type.MAP) { - - } else if (fieldSchema.getType() == Schema.Type.ENUM) { - PulsarColumnMetadata columnMetadata = new PulsarColumnMetadata( - PulsarColumnMetadata.getColumnName(handleKeyValueType, fieldName), - convertType(fieldSchema.getType(), fieldSchema.getLogicalType()), - null, null, false, false, - fieldNames.toArray(new String[fieldNames.size()]), - positionIndices.toArray(new Integer[positionIndices.size()]), handleKeyValueType); - columnMetadataList.add(columnMetadata); - - } else if (fieldSchema.getType() == Schema.Type.FIXED) { - - } else { - log.error("Unknown column type: {}", fieldSchema); - } - return columnMetadataList; - } - - @VisibleForTesting - static Type convertType(Schema.Type avroType, LogicalType logicalType) { - switch (avroType) { - case BOOLEAN: - return BooleanType.BOOLEAN; - case INT: - if (logicalType == LogicalTypes.timeMillis()) { - return TIME; - } else if (logicalType == LogicalTypes.date()) { - return DATE; - } - return IntegerType.INTEGER; - case LONG: - if (logicalType == LogicalTypes.timestampMillis()) { - return TIMESTAMP; - } - return BigintType.BIGINT; - case FLOAT: - return RealType.REAL; - case DOUBLE: - return DoubleType.DOUBLE; - case BYTES: - return VarbinaryType.VARBINARY; - case STRING: - return VarcharType.VARCHAR; - case ENUM: - return VarcharType.VARCHAR; - default: - log.error("Cannot convert type: %s", avroType); - return null; - } - } - - @VisibleForTesting - static boolean isPrimitiveType(Schema.Type type) { - return Schema.Type.NULL == type - || Schema.Type.BOOLEAN == type - || Schema.Type.INT == type - || Schema.Type.LONG == type - || Schema.Type.FLOAT == type - || Schema.Type.DOUBLE == type - || Schema.Type.BYTES == type - || Schema.Type.STRING == type; - - } } diff --git a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarPrimitiveSchemaHandler.java b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarPrimitiveSchemaHandler.java deleted file mode 100644 index dda08a3..0000000 --- a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarPrimitiveSchemaHandler.java +++ /dev/null @@ -1,61 +0,0 @@ -/** - * 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.netty.buffer.ByteBuf; -import java.sql.Time; -import java.sql.Timestamp; -import java.util.Date; -import org.apache.pulsar.client.impl.schema.AbstractSchema; -import org.apache.pulsar.client.impl.schema.AutoConsumeSchema; -import org.apache.pulsar.common.schema.SchemaInfo; - -/** - * A presto schema handler that interprets data using pulsar schema. - */ -public class PulsarPrimitiveSchemaHandler implements SchemaHandler { - - private final SchemaInfo schemaInfo; - private final AbstractSchema schema; - - PulsarPrimitiveSchemaHandler(SchemaInfo schemaInfo) { - this.schemaInfo = schemaInfo; - this.schema = (AbstractSchema) AutoConsumeSchema.getSchema(schemaInfo); - } - - @Override - public Object deserialize(ByteBuf payload) { - Object currentRecord = schema.decode(payload); - switch (schemaInfo.getType()) { - case DATE: - return ((Date) currentRecord).getTime(); - case TIME: - return ((Time) currentRecord).getTime(); - case TIMESTAMP: - return ((Timestamp) currentRecord).getTime(); - default: - return currentRecord; - } - } - - @Override - public Object extractField(int index, Object currentRecord) { - return currentRecord; - } -} diff --git a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarRecordCursor.java b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarRecordCursor.java index ac06d4c..c782de6 100644 --- a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarRecordCursor.java +++ b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarRecordCursor.java @@ -19,31 +19,30 @@ package org.apache.pulsar.sql.presto; import static com.google.common.base.Preconditions.checkArgument; -import static io.prestosql.spi.StandardErrorCode.NOT_SUPPORTED; -import static io.prestosql.spi.type.BigintType.BIGINT; -import static io.prestosql.spi.type.DateTimeEncoding.packDateTimeWithZone; -import static io.prestosql.spi.type.DateType.DATE; -import static io.prestosql.spi.type.IntegerType.INTEGER; -import static io.prestosql.spi.type.RealType.REAL; -import static io.prestosql.spi.type.SmallintType.SMALLINT; -import static io.prestosql.spi.type.TimeType.TIME; -import static io.prestosql.spi.type.TimestampType.TIMESTAMP; -import static io.prestosql.spi.type.TimestampWithTimeZoneType.TIMESTAMP_WITH_TIME_ZONE; -import static io.prestosql.spi.type.TinyintType.TINYINT; +import static com.google.common.collect.ImmutableSet.toImmutableSet; +import static io.prestosql.decoder.FieldValueProviders.bytesValueProvider; +import static io.prestosql.decoder.FieldValueProviders.longValueProvider; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.annotations.VisibleForTesting; import io.airlift.log.Logger; import io.airlift.slice.Slice; -import io.airlift.slice.Slices; import io.netty.buffer.ByteBuf; -import io.prestosql.spi.PrestoException; +import io.prestosql.decoder.DecoderColumnHandle; +import io.prestosql.decoder.FieldValueProvider; +import io.prestosql.spi.block.Block; +import io.prestosql.spi.connector.ColumnHandle; import io.prestosql.spi.connector.RecordCursor; import io.prestosql.spi.type.Type; -import io.prestosql.spi.type.VarbinaryType; -import io.prestosql.spi.type.VarcharType; import java.io.IOException; -import java.nio.ByteBuffer; +import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Objects; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.ExecutionException; import java.util.concurrent.atomic.AtomicLong; import org.apache.bookkeeper.mledger.AsyncCallbacks; import org.apache.bookkeeper.mledger.Entry; @@ -54,15 +53,21 @@ import org.apache.bookkeeper.mledger.ReadOnlyCursor; import org.apache.bookkeeper.mledger.impl.PositionImpl; import org.apache.bookkeeper.mledger.impl.ReadOnlyCursorImpl; +import org.apache.pulsar.client.api.PulsarClientException; +import org.apache.pulsar.client.impl.schema.KeyValueSchemaInfo; import org.apache.pulsar.common.api.raw.MessageParser; import org.apache.pulsar.common.api.raw.RawMessage; import org.apache.pulsar.common.naming.NamespaceName; import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.policies.data.OffloadPolicies; -import org.apache.pulsar.sql.presto.PulsarInternalColumn.PartitionColumn; +import org.apache.pulsar.common.schema.KeyValue; +import org.apache.pulsar.common.schema.KeyValueEncodingType; +import org.apache.pulsar.common.schema.SchemaInfo; +import org.apache.pulsar.common.schema.SchemaType; import org.jctools.queues.MessagePassingQueue; import org.jctools.queues.SpscArrayQueue; + /** * Implementation of a cursor to read records. */ @@ -74,10 +79,7 @@ public class PulsarRecordCursor implements RecordCursor { private ReadOnlyCursor cursor; private SpscArrayQueue messageQueue; private SpscArrayQueue entryQueue; - private Object currentRecord; private RawMessage currentMessage; - private Map internalColumnMap = PulsarInternalColumn.getInternalFieldsMap(); - private SchemaHandler schemaHandler; private int maxBatchSize; private long completedBytes = 0; private ReadEntries readEntries; @@ -97,10 +99,17 @@ public class PulsarRecordCursor implements RecordCursor { private int partition = -1; + private PulsarSqlSchemaInfoProvider schemaInfoProvider; + + private FieldValueProvider[] currentRowValues = null; + + PulsarDispatchingRowDecoderFactory decoderFactory; + private static final Logger log = Logger.get(PulsarRecordCursor.class); public PulsarRecordCursor(List columnHandles, PulsarSplit pulsarSplit, - PulsarConnectorConfig pulsarConnectorConfig) { + PulsarConnectorConfig pulsarConnectorConfig, + PulsarDispatchingRowDecoderFactory decoderFactory) { this.splitSize = pulsarSplit.getSplitSize(); // Set start time for split this.startTime = System.nanoTime(); @@ -126,21 +135,25 @@ public PulsarRecordCursor(List columnHandles, PulsarSplit pu pulsarSplit.getTableName()).getNamespaceObject(), offloadPolicies, pulsarConnectorConfig), new PulsarConnectorMetricsTracker(pulsarConnectorCache.getStatsProvider())); + this.decoderFactory = decoderFactory; } // Exposed for testing purposes PulsarRecordCursor(List columnHandles, PulsarSplit pulsarSplit, PulsarConnectorConfig - pulsarConnectorConfig, ManagedLedgerFactory managedLedgerFactory, ManagedLedgerConfig managedLedgerConfig, - PulsarConnectorMetricsTracker pulsarConnectorMetricsTracker) { + pulsarConnectorConfig, ManagedLedgerFactory managedLedgerFactory, ManagedLedgerConfig managedLedgerConfig, + PulsarConnectorMetricsTracker pulsarConnectorMetricsTracker, + PulsarDispatchingRowDecoderFactory decoderFactory) { this.splitSize = pulsarSplit.getSplitSize(); initialize(columnHandles, pulsarSplit, pulsarConnectorConfig, managedLedgerFactory, managedLedgerConfig, pulsarConnectorMetricsTracker); + this.decoderFactory = decoderFactory; } private void initialize(List columnHandles, PulsarSplit pulsarSplit, PulsarConnectorConfig pulsarConnectorConfig, ManagedLedgerFactory managedLedgerFactory, ManagedLedgerConfig managedLedgerConfig, - PulsarConnectorMetricsTracker pulsarConnectorMetricsTracker) { + PulsarConnectorMetricsTracker pulsarConnectorMetricsTracker) { this.columnHandles = columnHandles; + this.currentRowValues = new FieldValueProvider[columnHandles.size()]; this.pulsarSplit = pulsarSplit; this.partition = TopicName.getPartitionIndex(pulsarSplit.getTableName()); this.pulsarConnectorConfig = pulsarConnectorConfig; @@ -154,10 +167,14 @@ private void initialize(List columnHandles, PulsarSplit puls this.readOffloaded = pulsarConnectorConfig.getManagedLedgerOffloadDriver() != null; this.pulsarConnectorConfig = pulsarConnectorConfig; - this.schemaHandler = PulsarSchemaHandlers - .newPulsarSchemaHandler(this.topicName, - this.pulsarConnectorConfig, pulsarSplit.getSchemaInfo(), columnHandles); + try { + this.schemaInfoProvider = new PulsarSqlSchemaInfoProvider(this.topicName, + pulsarConnectorConfig.getPulsarAdmin()); + } catch (PulsarClientException e) { + log.error(e, "Failed to init Pulsar SchemaInfo Provider"); + throw new RuntimeException(e); + } log.info("Initializing split with parameters: %s", pulsarSplit); try { @@ -196,6 +213,11 @@ public Type getType(int field) { return columnHandles.get(field).getType(); } + @VisibleForTesting + public void setPulsarSqlSchemaInfoProvider(PulsarSqlSchemaInfoProvider schemaInfoProvider) { + this.schemaInfoProvider = schemaInfoProvider; + } + @VisibleForTesting class DeserializeEntries implements Runnable { @@ -419,148 +441,161 @@ public boolean advanceNextPosition() { //start time for deseralizing record metricsTracker.start_RECORD_DESERIALIZE_TIME(); - if (this.schemaHandler instanceof KeyValueSchemaHandler) { - ByteBuf keyByteBuf = null; - if (this.currentMessage.getKeyBytes().isPresent()) { - keyByteBuf = this.currentMessage.getKeyBytes().get(); - } - currentRecord = this.schemaHandler.deserialize(keyByteBuf, - this.currentMessage.getData(), this.currentMessage.getSchemaVersion()); - } else { - currentRecord = this.schemaHandler.deserialize(this.currentMessage.getData(), - this.currentMessage.getSchemaVersion()); + SchemaInfo schemaInfo; + try { + schemaInfo = schemaInfoProvider.getSchemaByVersion(this.currentMessage.getSchemaVersion()).get(); + } catch (InterruptedException | ExecutionException e) { + throw new RuntimeException(e); } - metricsTracker.incr_NUM_RECORD_DESERIALIZED(); - // stats for time spend deserializing - metricsTracker.end_RECORD_DESERIALIZE_TIME(); + Map currentRowValuesMap = new HashMap<>(); - return true; - } + if (schemaInfo.getType().equals(SchemaType.KEY_VALUE)) { + ByteBuf keyByteBuf; + ByteBuf valueByteBuf; + KeyValueEncodingType keyValueEncodingType = KeyValueSchemaInfo.decodeKeyValueEncodingType(schemaInfo); + if (Objects.equals(keyValueEncodingType, KeyValueEncodingType.INLINE)) { + ByteBuf dataPayload = this.currentMessage.getData(); + int keyLength = dataPayload.readInt(); + keyByteBuf = dataPayload.readSlice(keyLength); + int valueLength = dataPayload.readInt(); + valueByteBuf = dataPayload.readSlice(valueLength); + } else { + keyByteBuf = this.currentMessage.getKeyBytes().get(); + valueByteBuf = this.currentMessage.getData(); + } - @VisibleForTesting - Object getRecord(int fieldIndex) { - if (this.currentRecord == null) { - return null; - } + KeyValue kvSchemaInfo = KeyValueSchemaInfo.decodeKeyValueSchemaInfo(schemaInfo); + Set keyColumnHandles = columnHandles.stream() + .filter(col -> !col.isInternal()) + .filter(col -> PulsarColumnHandle.HandleKeyValueType.KEY + .equals(col.getHandleKeyValueType())) + .collect(toImmutableSet()); + PulsarRowDecoder keyDecoder = null; + if (keyColumnHandles.size() > 0) { + keyDecoder = decoderFactory.createRowDecoder(topicName, + kvSchemaInfo.getKey(), keyColumnHandles + ); + } - Object data; - PulsarColumnHandle pulsarColumnHandle = this.columnHandles.get(fieldIndex); + Set valueColumnHandles = columnHandles.stream() + .filter(col -> !col.isInternal()) + .filter(col -> PulsarColumnHandle.HandleKeyValueType.VALUE + .equals(col.getHandleKeyValueType())) + .collect(toImmutableSet()); + PulsarRowDecoder valueDecoder = null; + if (valueColumnHandles.size() > 0) { + valueDecoder = decoderFactory.createRowDecoder(topicName, + kvSchemaInfo.getValue(), + valueColumnHandles); + } - if (pulsarColumnHandle.isInternal()) { - String fieldName = this.columnHandles.get(fieldIndex).getName(); - PulsarInternalColumn pulsarInternalColumn = this.internalColumnMap.get(fieldName); - if (pulsarInternalColumn instanceof PartitionColumn) { - data = this.partition; - } else { - data = pulsarInternalColumn.getData(this.currentMessage); + Optional> decodedKey; + if (keyColumnHandles.size() > 0) { + decodedKey = keyDecoder.decodeRow(keyByteBuf); + decodedKey.ifPresent(currentRowValuesMap::putAll); + } + if (valueColumnHandles.size() > 0) { + Optional> decodedValue = + valueDecoder.decodeRow(valueByteBuf); + decodedValue.ifPresent(currentRowValuesMap::putAll); } } else { - data = this.schemaHandler.extractField(fieldIndex, this.currentRecord); + PulsarRowDecoder messageDecoder = decoderFactory.createRowDecoder(topicName, + schemaInfo, + columnHandles.stream() + .filter(col -> !col.isInternal()) + .filter(col -> PulsarColumnHandle.HandleKeyValueType.NONE + .equals(col.getHandleKeyValueType())) + .collect(toImmutableSet())); + Optional> decodedValue = + messageDecoder.decodeRow(this.currentMessage.getData()); + decodedValue.ifPresent(currentRowValuesMap::putAll); } - return data; + for (DecoderColumnHandle columnHandle : columnHandles) { + if (columnHandle.isInternal()) { + if (PulsarInternalColumn.PARTITION.getName().equals(columnHandle.getName())) { + currentRowValuesMap.put(columnHandle, longValueProvider(this.partition)); + } else if (PulsarInternalColumn.EVENT_TIME.getName().equals(columnHandle.getName())) { + currentRowValuesMap.put(columnHandle, PulsarFieldValueProviders.timeValueProvider( + this.currentMessage.getEventTime(), this.currentMessage.getPublishTime() == 0)); + } else if (PulsarInternalColumn.PUBLISH_TIME.getName().equals(columnHandle.getName())) { + currentRowValuesMap.put(columnHandle, PulsarFieldValueProviders.timeValueProvider( + this.currentMessage.getPublishTime(), this.currentMessage.getPublishTime() == 0)); + } else if (PulsarInternalColumn.MESSAGE_ID.getName().equals(columnHandle.getName())) { + currentRowValuesMap.put(columnHandle, bytesValueProvider( + this.currentMessage.getMessageId().toString().getBytes())); + } else if (PulsarInternalColumn.SEQUENCE_ID.getName().equals(columnHandle.getName())) { + currentRowValuesMap.put(columnHandle, longValueProvider(this.currentMessage.getSequenceId())); + } else if (PulsarInternalColumn.PRODUCER_NAME.getName().equals(columnHandle.getName())) { + currentRowValuesMap.put(columnHandle, + bytesValueProvider(this.currentMessage.getProducerName().getBytes())); + } else if (PulsarInternalColumn.KEY.getName().equals(columnHandle.getName())) { + String key = this.currentMessage.getKey().orElse(null); + currentRowValuesMap.put(columnHandle, bytesValueProvider(key == null ? null : key.getBytes())); + } else if (PulsarInternalColumn.PROPERTIES.getName().equals(columnHandle.getName())) { + try { + currentRowValuesMap.put(columnHandle, bytesValueProvider( + new ObjectMapper().writeValueAsBytes(this.currentMessage.getProperties()))); + } catch (JsonProcessingException e) { + throw new RuntimeException(e); + } + } else { + throw new IllegalArgumentException("unknown internal field " + columnHandle.getName()); + } + } + } + for (int i = 0; i < columnHandles.size(); i++) { + ColumnHandle columnHandle = columnHandles.get(i); + currentRowValues[i] = currentRowValuesMap.get(columnHandle); + } + + metricsTracker.incr_NUM_RECORD_DESERIALIZED(); + + // stats for time spend deserializing + metricsTracker.end_RECORD_DESERIALIZE_TIME(); + + return true; } + @Override public boolean getBoolean(int field) { - checkFieldType(field, boolean.class); - return (boolean) getRecord(field); + return getFieldValueProvider(field, boolean.class).getBoolean(); } @Override public long getLong(int field) { - checkFieldType(field, long.class); - - Object record = getRecord(field); - Type type = getType(field); - - if (type.equals(BIGINT)) { - return ((Number) record).longValue(); - } else if (type.equals(DATE)) { - return ((Number) record).longValue(); - } else if (type.equals(INTEGER)) { - return ((Number) record).intValue(); - } else if (type.equals(REAL)) { - return Float.floatToIntBits(((Number) record).floatValue()); - } else if (type.equals(SMALLINT)) { - return ((Number) record).shortValue(); - } else if (type.equals(TIME)) { - return ((Number) record).longValue(); - } else if (type.equals(TIMESTAMP)) { - if (record instanceof String) { - return Long.parseLong((String) record); - } else { - return ((Number) record).longValue(); - } - } else if (type.equals(TIMESTAMP_WITH_TIME_ZONE)) { - return packDateTimeWithZone(((Number) record).longValue(), 0); - } else if (type.equals(TINYINT)) { - return Byte.parseByte(record.toString()); - } else { - throw new PrestoException(NOT_SUPPORTED, "Unsupported type " + getType(field)); - } + return getFieldValueProvider(field, long.class).getLong(); } @Override public double getDouble(int field) { - checkFieldType(field, double.class); - Object record = getRecord(field); - return (double) record; + return getFieldValueProvider(field, double.class).getDouble(); } @Override public Slice getSlice(int field) { - checkFieldType(field, Slice.class); - - Object record = getRecord(field); - Type type = getType(field); - if (type == VarcharType.VARCHAR) { - return Slices.utf8Slice(record.toString()); - } else if (type == VarbinaryType.VARBINARY) { - return Slices.wrappedBuffer(toBytes(record)); - } else { - throw new PrestoException(NOT_SUPPORTED, "Unsupported type " + type); - } + return getFieldValueProvider(field, Slice.class).getSlice(); } - private byte[] toBytes(Object record) { - if (record instanceof ByteBuffer) { - ByteBuffer byteBuffer = (ByteBuffer) record; - if (byteBuffer.hasArray()) { - return byteBuffer.array(); - } - byte[] bytes = new byte[byteBuffer.position()]; - byteBuffer.flip(); - byteBuffer.get(bytes); - return bytes; - } else if (record instanceof ByteBuf) { - ByteBuf byteBuf = (ByteBuf) record; - if (byteBuf.hasArray()) { - return byteBuf.array(); - } - byte[] bytes = new byte[byteBuf.readableBytes()]; - byteBuf.readBytes(bytes); - return bytes; - } else { - try { - return (byte[]) record; - } catch (Exception e) { - throw new PrestoException(NOT_SUPPORTED, "Unsupported type " + record.getClass().getName()); - } - } + private FieldValueProvider getFieldValueProvider(int fieldIndex, Class expectedType) { + checkArgument(fieldIndex < columnHandles.size(), "Invalid field index"); + checkFieldType(fieldIndex, expectedType); + return currentRowValues[fieldIndex]; } @Override public Object getObject(int field) { - throw new UnsupportedOperationException(); + return getFieldValueProvider(field, Block.class).getBlock(); } @Override public boolean isNull(int field) { - Object record = getRecord(field); - return record == null; + FieldValueProvider provider = currentRowValues[field]; + return provider == null || provider.isNull(); } @Override @@ -604,8 +639,4 @@ private void checkFieldType(int field, Class expected) { checkArgument(actual == expected, "Expected field %s to be type %s but is %s", field, expected, actual); } - @VisibleForTesting - SchemaHandler getSchemaHandler() { - return this.schemaHandler; - } } diff --git a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarRecordSet.java b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarRecordSet.java index 20d10c1..d9595ea 100644 --- a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarRecordSet.java +++ b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarRecordSet.java @@ -23,6 +23,7 @@ import io.prestosql.spi.connector.RecordCursor; import io.prestosql.spi.connector.RecordSet; import io.prestosql.spi.type.Type; + import java.util.List; /** @@ -35,8 +36,10 @@ public class PulsarRecordSet implements RecordSet { private final PulsarSplit pulsarSplit; private final PulsarConnectorConfig pulsarConnectorConfig; + private PulsarDispatchingRowDecoderFactory decoderFactory; + public PulsarRecordSet(PulsarSplit split, List columnHandles, PulsarConnectorConfig - pulsarConnectorConfig) { + pulsarConnectorConfig, PulsarDispatchingRowDecoderFactory decoderFactory) { requireNonNull(split, "split is null"); this.columnHandles = requireNonNull(columnHandles, "column handles is null"); ImmutableList.Builder types = ImmutableList.builder(); @@ -48,6 +51,8 @@ public PulsarRecordSet(PulsarSplit split, List columnHandles this.pulsarSplit = split; this.pulsarConnectorConfig = pulsarConnectorConfig; + + this.decoderFactory = decoderFactory; } @@ -58,6 +63,7 @@ public List getColumnTypes() { @Override public RecordCursor cursor() { - return new PulsarRecordCursor(this.columnHandles, this.pulsarSplit, this.pulsarConnectorConfig); + return new PulsarRecordCursor(this.columnHandles, this.pulsarSplit, + this.pulsarConnectorConfig, this.decoderFactory); } } diff --git a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarRecordSetProvider.java b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarRecordSetProvider.java index a15187a..cf69810 100644 --- a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarRecordSetProvider.java +++ b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarRecordSetProvider.java @@ -36,8 +36,12 @@ public class PulsarRecordSetProvider implements ConnectorRecordSetProvider { private final PulsarConnectorConfig pulsarConnectorConfig; + private final PulsarDispatchingRowDecoderFactory decoderFactory; + @Inject - public PulsarRecordSetProvider(PulsarConnectorConfig pulsarConnectorConfig) { + public PulsarRecordSetProvider(PulsarConnectorConfig pulsarConnectorConfig, + PulsarDispatchingRowDecoderFactory decoderFactory) { + this.decoderFactory = requireNonNull(decoderFactory, "decoderFactory is null"); this.pulsarConnectorConfig = requireNonNull(pulsarConnectorConfig, "pulsarConnectorConfig is null"); } @@ -53,6 +57,6 @@ public RecordSet getRecordSet(ConnectorTransactionHandle transactionHandle, Conn handles.add((PulsarColumnHandle) handle); } - return new PulsarRecordSet(pulsarSplit, handles.build(), this.pulsarConnectorConfig); + return new PulsarRecordSet(pulsarSplit, handles.build(), this.pulsarConnectorConfig, decoderFactory); } } diff --git a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/SchemaHandler.java b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarRowDecoder.java similarity index 54% rename from pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/SchemaHandler.java rename to pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarRowDecoder.java index 050eec9..602ec37 100644 --- a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/SchemaHandler.java +++ b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarRowDecoder.java @@ -19,27 +19,23 @@ package org.apache.pulsar.sql.presto; import io.netty.buffer.ByteBuf; +import io.prestosql.decoder.DecoderColumnHandle; +import io.prestosql.decoder.FieldValueProvider; + +import java.util.Map; +import java.util.Optional; /** - * This interface defines the methods to work with schemas. + * RowDecoder interface for Pulsar. */ -public interface SchemaHandler { - - default Object deserialize(ByteBuf payload) { - return null; - } - - // if schemaHandler don't support multi version, we will use method deserialize(ByteBuf payload) - default Object deserialize(ByteBuf byteBuf, byte[] schemaVersion) { - return deserialize(byteBuf); - } - - // if SchemaHandler don't support key value multi version - // we will use method deserialize(ByteBuf byteBuf, byte[] schemaVersion) - default Object deserialize(ByteBuf keyPayload, ByteBuf dataPayload, byte[] schemaVersion) { - return deserialize(dataPayload, schemaVersion); - } +public interface PulsarRowDecoder { - Object extractField(int index, Object currentRecord); + /** + * decode byteBuf to Column FieldValueProvider. + * + * @param byteBuf + * @return + */ + Optional> decodeRow(ByteBuf byteBuf); } diff --git a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarRowDecoderFactory.java b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarRowDecoderFactory.java new file mode 100644 index 0000000..c2a2b73 --- /dev/null +++ b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarRowDecoderFactory.java @@ -0,0 +1,51 @@ +/** + * 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.prestosql.decoder.DecoderColumnHandle; +import io.prestosql.spi.connector.ColumnMetadata; +import java.util.List; +import java.util.Set; +import org.apache.pulsar.common.naming.TopicName; +import org.apache.pulsar.common.schema.SchemaInfo; + +/** + * Pulsar customized RowDecoderFactory interface. + */ +public interface PulsarRowDecoderFactory { + + /** + * extract ColumnMetadata from pulsar SchemaInfo and HandleKeyValueType. + * @param schemaInfo + * @param handleKeyValueType + * @return + */ + List extractColumnMetadata(TopicName topicName, SchemaInfo schemaInfo, + PulsarColumnHandle.HandleKeyValueType handleKeyValueType); + + /** + * createRowDecoder RowDecoder by pulsar SchemaInfo and column DecoderColumnHandles. + * @param schemaInfo + * @param columns + * @return + */ + PulsarRowDecoder createRowDecoder(TopicName topicName, SchemaInfo schemaInfo, + Set columns); + +} diff --git a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarSchemaHandlers.java b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarSchemaHandlers.java deleted file mode 100644 index 05137e6..0000000 --- a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarSchemaHandlers.java +++ /dev/null @@ -1,67 +0,0 @@ -/** - * 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 static io.prestosql.spi.StandardErrorCode.NOT_SUPPORTED; -import io.prestosql.spi.PrestoException; -import java.util.List; -import org.apache.pulsar.client.api.PulsarClientException; -import org.apache.pulsar.client.api.Schema; -import org.apache.pulsar.common.naming.TopicName; -import org.apache.pulsar.common.schema.SchemaInfo; -import org.apache.pulsar.common.schema.SchemaType; - -class PulsarSchemaHandlers { - - static SchemaHandler newPulsarSchemaHandler(TopicName topicName, - PulsarConnectorConfig pulsarConnectorConfig, - SchemaInfo schemaInfo, - List columnHandles) throws RuntimeException{ - if (schemaInfo.getType().isPrimitive()) { - return new PulsarPrimitiveSchemaHandler(schemaInfo); - } else if (schemaInfo.getType().isStruct()) { - try { - switch (schemaInfo.getType()) { - case JSON: - return new JSONSchemaHandler(columnHandles); - case AVRO: - return new AvroSchemaHandler(topicName, pulsarConnectorConfig, schemaInfo, columnHandles); - default: - throw new PrestoException(NOT_SUPPORTED, "Not supported schema type: " + schemaInfo.getType()); - } - } catch (PulsarClientException e) { - throw new RuntimeException( - new Throwable("PulsarAdmin gets version schema fail, topicName : " - + topicName.toString(), e)); - } - } else if (schemaInfo.getType().equals(SchemaType.KEY_VALUE)) { - return new KeyValueSchemaHandler(topicName, pulsarConnectorConfig, schemaInfo, columnHandles); - } else { - throw new PrestoException( - NOT_SUPPORTED, - "Schema `" + schemaInfo.getType() + "` is not supported by presto yet : " + schemaInfo); - } - - } - - static SchemaInfo defaultSchema() { - return Schema.BYTES.getSchemaInfo(); - } -} diff --git a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarSplitManager.java b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarSplitManager.java index 3485f98..4db7eb2 100644 --- a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarSplitManager.java +++ b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarSplitManager.java @@ -116,7 +116,7 @@ public ConnectorSplitSource getSplits(ConnectorTransactionHandle transactionHand String.format("Failed to get pulsar topic schema for topic %s/%s: Unauthorized", namespace, tableHandle.getTableName())); } else if (e.getStatusCode() == 404) { - schemaInfo = PulsarSchemaHandlers.defaultSchema(); + schemaInfo = PulsarSqlSchemaInfoProvider.defaultSchema(); } else { throw new RuntimeException("Failed to get pulsar topic schema for topic " + String.format("%s/%s", namespace, tableHandle.getTableName()) diff --git a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarSqlSchemaInfoProvider.java b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarSqlSchemaInfoProvider.java index 41e49c0..fe9b619 100644 --- a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarSqlSchemaInfoProvider.java +++ b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarSqlSchemaInfoProvider.java @@ -19,16 +19,20 @@ package org.apache.pulsar.sql.presto; import static java.util.concurrent.CompletableFuture.completedFuture; + import com.google.common.cache.CacheBuilder; import com.google.common.cache.CacheLoader; import com.google.common.cache.LoadingCache; + import java.nio.ByteBuffer; import java.nio.charset.StandardCharsets; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; + import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.admin.PulsarAdminException; +import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.client.api.schema.SchemaInfoProvider; import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.protocol.schema.BytesSchemaVersion; @@ -57,7 +61,7 @@ public SchemaInfo load(BytesSchemaVersion schemaVersion) throws Exception { } }); - PulsarSqlSchemaInfoProvider(TopicName topicName, PulsarAdmin pulsarAdmin) { + public PulsarSqlSchemaInfoProvider(TopicName topicName, PulsarAdmin pulsarAdmin) { this.topicName = topicName; this.pulsarAdmin = pulsarAdmin; } @@ -98,4 +102,9 @@ private SchemaInfo loadSchema(BytesSchemaVersion bytesSchemaVersion) throws Puls .getSchemaInfo(topicName.toString(), ByteBuffer.wrap(bytesSchemaVersion.get()).getLong()); } -} + + public static SchemaInfo defaultSchema(){ + return Schema.BYTES.getSchemaInfo(); + } + +} \ No newline at end of file diff --git a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/decoder/avro/PulsarAvroColumnDecoder.java b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/decoder/avro/PulsarAvroColumnDecoder.java new file mode 100644 index 0000000..3b7e293 --- /dev/null +++ b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/decoder/avro/PulsarAvroColumnDecoder.java @@ -0,0 +1,385 @@ +/** + * 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.decoder.avro; + +import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.base.Preconditions.checkState; +import static io.airlift.slice.Slices.utf8Slice; +import static io.prestosql.decoder.DecoderErrorCode.DECODER_CONVERSION_NOT_SUPPORTED; +import static io.prestosql.spi.StandardErrorCode.GENERIC_USER_ERROR; +import static io.prestosql.spi.type.Varchars.truncateToLength; +import static java.lang.Float.floatToIntBits; +import static java.lang.String.format; +import static java.util.Objects.requireNonNull; + +import com.google.common.base.Splitter; +import com.google.common.collect.ImmutableSet; +import io.airlift.slice.Slice; +import io.airlift.slice.Slices; +import io.prestosql.decoder.DecoderColumnHandle; +import io.prestosql.decoder.FieldValueProvider; +import io.prestosql.spi.PrestoException; +import io.prestosql.spi.block.Block; +import io.prestosql.spi.block.BlockBuilder; +import io.prestosql.spi.type.ArrayType; +import io.prestosql.spi.type.BigintType; +import io.prestosql.spi.type.BooleanType; +import io.prestosql.spi.type.DateType; +import io.prestosql.spi.type.DoubleType; +import io.prestosql.spi.type.IntegerType; +import io.prestosql.spi.type.MapType; +import io.prestosql.spi.type.RealType; +import io.prestosql.spi.type.RowType; +import io.prestosql.spi.type.RowType.Field; +import io.prestosql.spi.type.SmallintType; +import io.prestosql.spi.type.TimeType; +import io.prestosql.spi.type.TimestampType; +import io.prestosql.spi.type.TinyintType; +import io.prestosql.spi.type.Type; +import io.prestosql.spi.type.VarbinaryType; +import io.prestosql.spi.type.VarcharType; +import java.nio.ByteBuffer; + +import java.util.List; +import java.util.Map; +import java.util.Set; + +import org.apache.avro.generic.GenericEnumSymbol; +import org.apache.avro.generic.GenericFixed; +import org.apache.avro.generic.GenericRecord; + +/** + * Copy from {@link io.prestosql.decoder.avro.AvroColumnDecoder} (presto-record-decoder-345) + * with A little bit pulsar's extensions. + * 1) support {@link io.prestosql.spi.type.TimestampType},{@link io.prestosql.spi.type.DateType}DATE, + * * {@link io.prestosql.spi.type.TimeType}. + * 2) support {@link io.prestosql.spi.type.RealType}. + */ +public class PulsarAvroColumnDecoder { + private static final Set SUPPORTED_PRIMITIVE_TYPES = ImmutableSet.of( + BooleanType.BOOLEAN, + TinyintType.TINYINT, + SmallintType.SMALLINT, + IntegerType.INTEGER, + BigintType.BIGINT, + RealType.REAL, + DoubleType.DOUBLE, + TimestampType.TIMESTAMP, + DateType.DATE, + TimeType.TIME, + VarbinaryType.VARBINARY); + + private final Type columnType; + private final String columnMapping; + private final String columnName; + + public PulsarAvroColumnDecoder(DecoderColumnHandle columnHandle) { + try { + requireNonNull(columnHandle, "columnHandle is null"); + this.columnType = columnHandle.getType(); + this.columnMapping = columnHandle.getMapping(); + this.columnName = columnHandle.getName(); + checkArgument(!columnHandle.isInternal(), + "unexpected internal column '%s'", columnName); + checkArgument(columnHandle.getFormatHint() == null, + "unexpected format hint '%s' defined for column '%s'", columnHandle.getFormatHint(), columnName); + checkArgument(columnHandle.getDataFormat() == null, + "unexpected data format '%s' defined for column '%s'", columnHandle.getDataFormat(), columnName); + checkArgument(columnHandle.getMapping() != null, + "mapping not defined for column '%s'", columnName); + checkArgument(isSupportedType(columnType), + "Unsupported column type '%s' for column '%s'", columnType, columnName); + } catch (IllegalArgumentException e) { + throw new PrestoException(GENERIC_USER_ERROR, e); + } + } + + private boolean isSupportedType(Type type) { + if (isSupportedPrimitive(type)) { + return true; + } + + if (type instanceof ArrayType) { + checkArgument(type.getTypeParameters().size() == 1, + "expecting exactly one type parameter for array"); + return isSupportedType(type.getTypeParameters().get(0)); + } + + if (type instanceof MapType) { + List typeParameters = type.getTypeParameters(); + checkArgument(typeParameters.size() == 2, + "expecting exactly two type parameters for map"); + checkArgument(typeParameters.get(0) instanceof VarcharType, + "Unsupported column type '%s' for map key", typeParameters.get(0)); + return isSupportedType(type.getTypeParameters().get(1)); + } + + if (type instanceof RowType) { + for (Type fieldType : type.getTypeParameters()) { + if (!isSupportedType(fieldType)) { + return false; + } + } + return true; + } + return false; + } + + private boolean isSupportedPrimitive(Type type) { + return type instanceof VarcharType || SUPPORTED_PRIMITIVE_TYPES.contains(type); + } + + public FieldValueProvider decodeField(GenericRecord avroRecord) { + Object avroColumnValue = locateNode(avroRecord, columnMapping); + return new ObjectValueProvider(avroColumnValue, columnType, columnName); + } + + private static Object locateNode(GenericRecord element, String columnMapping) { + Object value = element; + for (String pathElement : Splitter.on('/').omitEmptyStrings().split(columnMapping)) { + if (value == null) { + return null; + } + value = ((GenericRecord) value).get(pathElement); + } + return value; + } + + private static class ObjectValueProvider + extends FieldValueProvider { + private final Object value; + private final Type columnType; + private final String columnName; + + public ObjectValueProvider(Object value, Type columnType, String columnName) { + this.value = value; + this.columnType = columnType; + this.columnName = columnName; + } + + @Override + public boolean isNull() { + return value == null; + } + + @Override + public double getDouble() { + if (value instanceof Double || value instanceof Float) { + return ((Number) value).doubleValue(); + } + throw new PrestoException(DECODER_CONVERSION_NOT_SUPPORTED, + format("cannot decode object of '%s' as '%s' for column '%s'", + value.getClass(), columnType, columnName)); + } + + @Override + public boolean getBoolean() { + if (value instanceof Boolean) { + return (Boolean) value; + } + throw new PrestoException(DECODER_CONVERSION_NOT_SUPPORTED, + format("cannot decode object of '%s' as '%s' for column '%s'", + value.getClass(), columnType, columnName)); + } + + @Override + public long getLong() { + if (value instanceof Long || value instanceof Integer) { + return ((Number) value).longValue(); + } + + if (columnType instanceof RealType) { + return floatToIntBits((Float) value); + } + + throw new PrestoException(DECODER_CONVERSION_NOT_SUPPORTED, + format("cannot decode object of '%s' as '%s' for column '%s'", + value.getClass(), columnType, columnName)); + } + + @Override + public Slice getSlice() { + return PulsarAvroColumnDecoder.getSlice(value, columnType, columnName); + } + + @Override + public Block getBlock() { + return serializeObject(null, value, columnType, columnName); + } + } + + private static Slice getSlice(Object value, Type type, String columnName) { + if (type instanceof VarcharType && (value instanceof CharSequence || value instanceof GenericEnumSymbol)) { + return truncateToLength(utf8Slice(value.toString()), type); + } + + if (type instanceof VarbinaryType) { + if (value instanceof ByteBuffer) { + return Slices.wrappedBuffer((ByteBuffer) value); + } else if (value instanceof GenericFixed) { + return Slices.wrappedBuffer(((GenericFixed) value).bytes()); + } + } + + throw new PrestoException(DECODER_CONVERSION_NOT_SUPPORTED, + format("cannot decode object of '%s' as '%s' for column '%s'", + value.getClass(), type, columnName)); + } + + private static Block serializeObject(BlockBuilder builder, Object value, Type type, String columnName) { + if (type instanceof ArrayType) { + return serializeList(builder, value, type, columnName); + } + if (type instanceof MapType) { + return serializeMap(builder, value, type, columnName); + } + if (type instanceof RowType) { + return serializeRow(builder, value, type, columnName); + } + serializePrimitive(builder, value, type, columnName); + return null; + } + + private static Block serializeList(BlockBuilder parentBlockBuilder, Object value, Type type, String columnName) { + if (value == null) { + checkState(parentBlockBuilder != null, "parentBlockBuilder is null"); + parentBlockBuilder.appendNull(); + return null; + } + List list = (List) value; + List typeParameters = type.getTypeParameters(); + Type elementType = typeParameters.get(0); + + BlockBuilder blockBuilder = elementType.createBlockBuilder(null, list.size()); + for (Object element : list) { + serializeObject(blockBuilder, element, elementType, columnName); + } + if (parentBlockBuilder != null) { + type.writeObject(parentBlockBuilder, blockBuilder.build()); + return null; + } + return blockBuilder.build(); + } + + private static void serializePrimitive(BlockBuilder blockBuilder, Object value, Type type, String columnName) { + requireNonNull(blockBuilder, "parent blockBuilder is null"); + + if (value == null) { + blockBuilder.appendNull(); + return; + } + + if (type instanceof BooleanType) { + type.writeBoolean(blockBuilder, (Boolean) value); + return; + } + + if ((value instanceof Integer || value instanceof Long) + && (type instanceof BigintType || type instanceof IntegerType + || type instanceof SmallintType || type instanceof TinyintType)) { + type.writeLong(blockBuilder, ((Number) value).longValue()); + return; + } + + if (type instanceof DoubleType) { + type.writeDouble(blockBuilder, (Double) value); + return; + } + + if (type instanceof RealType) { + type.writeLong(blockBuilder, floatToIntBits((Float) value)); + return; + } + + if (type instanceof VarcharType || type instanceof VarbinaryType) { + type.writeSlice(blockBuilder, getSlice(value, type, columnName)); + return; + } + + if (type instanceof TimestampType) { + type.writeLong(blockBuilder, (Long) value); + return; + } + + throw new PrestoException(DECODER_CONVERSION_NOT_SUPPORTED, + format("cannot decode object of '%s' as '%s' for column '%s'", + value.getClass(), type, columnName)); + } + + private static Block serializeMap(BlockBuilder parentBlockBuilder, Object value, Type type, String columnName) { + if (value == null) { + checkState(parentBlockBuilder != null, "parentBlockBuilder is null"); + parentBlockBuilder.appendNull(); + return null; + } + + Map map = (Map) value; + List typeParameters = type.getTypeParameters(); + Type keyType = typeParameters.get(0); + Type valueType = typeParameters.get(1); + + BlockBuilder blockBuilder; + if (parentBlockBuilder != null) { + blockBuilder = parentBlockBuilder; + } else { + blockBuilder = type.createBlockBuilder(null, 1); + } + + BlockBuilder entryBuilder = blockBuilder.beginBlockEntry(); + for (Map.Entry entry : map.entrySet()) { + if (entry.getKey() != null) { + keyType.writeSlice(entryBuilder, truncateToLength(utf8Slice(entry.getKey().toString()), keyType)); + serializeObject(entryBuilder, entry.getValue(), valueType, columnName); + } + } + blockBuilder.closeEntry(); + + if (parentBlockBuilder == null) { + return blockBuilder.getObject(0, Block.class); + } + return null; + } + + private static Block serializeRow(BlockBuilder parentBlockBuilder, Object value, Type type, String columnName) { + if (value == null) { + checkState(parentBlockBuilder != null, "parent block builder is null"); + parentBlockBuilder.appendNull(); + return null; + } + + BlockBuilder blockBuilder; + if (parentBlockBuilder != null) { + blockBuilder = parentBlockBuilder; + } else { + blockBuilder = type.createBlockBuilder(null, 1); + } + BlockBuilder singleRowBuilder = blockBuilder.beginBlockEntry(); + GenericRecord record = (GenericRecord) value; + List fields = ((RowType) type).getFields(); + for (Field field : fields) { + checkState(field.getName().isPresent(), "field name not found"); + serializeObject(singleRowBuilder, record.get(field.getName().get()), field.getType(), columnName); + } + blockBuilder.closeEntry(); + if (parentBlockBuilder == null) { + return blockBuilder.getObject(0, Block.class); + } + return null; + } +} diff --git a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/decoder/avro/PulsarAvroRowDecoder.java b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/decoder/avro/PulsarAvroRowDecoder.java new file mode 100644 index 0000000..b7c946a --- /dev/null +++ b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/decoder/avro/PulsarAvroRowDecoder.java @@ -0,0 +1,77 @@ +/** + * 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.decoder.avro; + +import static com.google.common.base.Functions.identity; +import static com.google.common.collect.ImmutableMap.toImmutableMap; +import static io.prestosql.spi.StandardErrorCode.GENERIC_INTERNAL_ERROR; +import static java.util.Objects.requireNonNull; + +import io.netty.buffer.ByteBuf; +import io.prestosql.decoder.DecoderColumnHandle; +import io.prestosql.decoder.FieldValueProvider; +import io.prestosql.spi.PrestoException; +import java.util.Map; +import java.util.Optional; +import java.util.Set; + +import org.apache.avro.generic.GenericRecord; +import org.apache.pulsar.client.impl.schema.generic.GenericAvroRecord; +import org.apache.pulsar.client.impl.schema.generic.GenericAvroSchema; +import org.apache.pulsar.sql.presto.PulsarRowDecoder; + +/** + * Refer to {@link io.prestosql.decoder.avro.AvroRowDecoder}. + */ +public class PulsarAvroRowDecoder implements PulsarRowDecoder { + + private final GenericAvroSchema genericAvroSchema; + private final Map columnDecoders; + + public PulsarAvroRowDecoder(GenericAvroSchema genericAvroSchema, Set columns) { + this.genericAvroSchema = requireNonNull(genericAvroSchema, "genericAvroSchema is null"); + columnDecoders = columns.stream() + .collect(toImmutableMap(identity(), this::createColumnDecoder)); + } + + private PulsarAvroColumnDecoder createColumnDecoder(DecoderColumnHandle columnHandle) { + return new PulsarAvroColumnDecoder(columnHandle); + } + + /** + * decode ByteBuf by {@link org.apache.pulsar.client.api.schema.GenericSchema}. + * @param byteBuf + * @return + */ + @Override + public Optional> decodeRow(ByteBuf byteBuf) { + GenericRecord avroRecord; + try { + GenericAvroRecord record = (GenericAvroRecord) genericAvroSchema.decode(byteBuf); + avroRecord = record.getAvroRecord(); + } catch (Exception e) { + e.printStackTrace(); + throw new PrestoException(GENERIC_INTERNAL_ERROR, "Decoding avro record failed.", e); + } + return Optional.of(columnDecoders.entrySet().stream() + .collect(toImmutableMap( + Map.Entry::getKey, + entry -> entry.getValue().decodeField(avroRecord)))); + } +} diff --git a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/decoder/avro/PulsarAvroRowDecoderFactory.java b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/decoder/avro/PulsarAvroRowDecoderFactory.java new file mode 100644 index 0000000..26c333a --- /dev/null +++ b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/decoder/avro/PulsarAvroRowDecoderFactory.java @@ -0,0 +1,190 @@ +/** + * 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.decoder.avro; + +import static com.google.common.collect.ImmutableList.toImmutableList; +import static io.prestosql.spi.StandardErrorCode.NOT_SUPPORTED; +import static io.prestosql.spi.type.DateType.DATE; +import static io.prestosql.spi.type.TimeType.TIME; +import static io.prestosql.spi.type.VarcharType.createUnboundedVarcharType; +import static java.lang.String.format; +import static java.util.stream.Collectors.toList; + +import com.google.common.collect.ImmutableList; +import io.airlift.log.Logger; +import io.prestosql.decoder.DecoderColumnHandle; +import io.prestosql.spi.PrestoException; +import io.prestosql.spi.connector.ColumnMetadata; +import io.prestosql.spi.type.ArrayType; +import io.prestosql.spi.type.BigintType; +import io.prestosql.spi.type.BooleanType; +import io.prestosql.spi.type.DoubleType; +import io.prestosql.spi.type.IntegerType; +import io.prestosql.spi.type.RealType; +import io.prestosql.spi.type.RowType; +import io.prestosql.spi.type.StandardTypes; +import io.prestosql.spi.type.TimestampType; +import io.prestosql.spi.type.Type; +import io.prestosql.spi.type.TypeManager; +import io.prestosql.spi.type.TypeSignature; +import io.prestosql.spi.type.TypeSignatureParameter; +import io.prestosql.spi.type.VarbinaryType; +import io.prestosql.spi.type.VarcharType; + +import java.util.List; +import java.util.Optional; +import java.util.Set; + +import org.apache.avro.LogicalType; +import org.apache.avro.LogicalTypes; +import org.apache.avro.Schema; +import org.apache.avro.SchemaParseException; +import org.apache.commons.lang3.StringUtils; +import org.apache.pulsar.client.impl.schema.generic.GenericAvroSchema; +import org.apache.pulsar.client.impl.schema.generic.GenericJsonSchema; +import org.apache.pulsar.common.naming.TopicName; +import org.apache.pulsar.common.schema.SchemaInfo; +import org.apache.pulsar.sql.presto.PulsarColumnHandle; +import org.apache.pulsar.sql.presto.PulsarColumnMetadata; +import org.apache.pulsar.sql.presto.PulsarRowDecoder; +import org.apache.pulsar.sql.presto.PulsarRowDecoderFactory; + +/** + * PulsarRowDecoderFactory for {@link org.apache.pulsar.common.schema.SchemaType#AVRO}. + */ +public class PulsarAvroRowDecoderFactory implements PulsarRowDecoderFactory { + + private TypeManager typeManager; + + public PulsarAvroRowDecoderFactory(TypeManager typeManager) { + this.typeManager = typeManager; + } + + @Override + public PulsarRowDecoder createRowDecoder(TopicName topicName, SchemaInfo schemaInfo, + Set columns) { + return new PulsarAvroRowDecoder((GenericAvroSchema) GenericAvroSchema.of(schemaInfo), columns); + } + + @Override + public List extractColumnMetadata(TopicName topicName, SchemaInfo schemaInfo, + PulsarColumnHandle.HandleKeyValueType handleKeyValueType) { + List columnMetadata; + String schemaJson = new String(schemaInfo.getSchema()); + if (StringUtils.isBlank(schemaJson)) { + throw new PrestoException(NOT_SUPPORTED, "Topic " + + topicName.toString() + " does not have a valid schema"); + } + Schema schema; + try { + schema = GenericJsonSchema.of(schemaInfo).getAvroSchema(); + } catch (SchemaParseException ex) { + throw new PrestoException(NOT_SUPPORTED, "Topic " + + topicName.toString() + " does not have a valid schema"); + } + + try { + columnMetadata = schema.getFields().stream() + .map(field -> + new PulsarColumnMetadata(PulsarColumnMetadata.getColumnName(handleKeyValueType, field.name()), parseAvroPrestoType( + field.name(), field.schema()), field.schema().toString(), null, false, false, + handleKeyValueType, new PulsarColumnMetadata.DecoderExtraInfo(field.name(), + null, null)) + + ).collect(toList()); + }catch (StackOverflowError e){ + log.warn(e, "Topic " + + topicName.toString() + " extractColumnMetadata failed."); + throw new PrestoException(NOT_SUPPORTED, "Topic " + + topicName.toString() + " schema may contains cyclic definitions.",e); + } + return columnMetadata; + } + + private Type parseAvroPrestoType(String fieldname, Schema schema) { + Schema.Type type = schema.getType(); + LogicalType logicalType = schema.getLogicalType(); + switch (type) { + case STRING: + case ENUM: + return createUnboundedVarcharType(); + case NULL: + throw new UnsupportedOperationException( + format("field '%s' NULL type code should not be reached," + + "please check the schema or report the bug.", fieldname)); + case FIXED: + case BYTES: + //TODO: support decimal logicalType + return VarbinaryType.VARBINARY; + case INT: + if (logicalType == LogicalTypes.timeMillis()) { + return TIME; + } else if (logicalType == LogicalTypes.date()) { + return DATE; + } + return IntegerType.INTEGER; + case LONG: + if (logicalType == LogicalTypes.timestampMillis()) { + return TimestampType.TIMESTAMP; + } + //TODO: support timestamp_microseconds logicalType : https://github.com/prestosql/presto/issues/1284 + return BigintType.BIGINT; + case FLOAT: + return RealType.REAL; + case DOUBLE: + return DoubleType.DOUBLE; + case BOOLEAN: + return BooleanType.BOOLEAN; + case ARRAY: + return new ArrayType(parseAvroPrestoType(fieldname, schema.getElementType())); + case MAP: + //The key for an avro map must be string + TypeSignature valueType = parseAvroPrestoType(fieldname, schema.getValueType()).getTypeSignature(); + return typeManager.getParameterizedType(StandardTypes.MAP, + ImmutableList.of(TypeSignatureParameter.typeParameter(VarcharType.VARCHAR.getTypeSignature()), + TypeSignatureParameter.typeParameter(valueType))); + case RECORD: + if (schema.getFields().size() > 0) { + return RowType.from(schema.getFields().stream() + .map(field -> new RowType.Field(Optional.of(field.name()), + parseAvroPrestoType(field.name(), field.schema()))) + .collect(toImmutableList())); + } else { + throw new UnsupportedOperationException(format( + "field '%s' of record type has no fields, " + + "please check schema definition. ", fieldname)); + } + case UNION: + for (Schema nestType : schema.getTypes()) { + if (nestType.getType() != Schema.Type.NULL) { + return parseAvroPrestoType(fieldname, nestType); + } + } + throw new UnsupportedOperationException(format( + "field '%s' of UNION type must contains not NULL type.", fieldname)); + default: + throw new UnsupportedOperationException(format( + "Can't convert from schema type '%s' (%s) to presto type.", + schema.getType(), schema.getFullName())); + } + } + + private static final Logger log = Logger.get(PulsarAvroRowDecoderFactory.class); + +} diff --git a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/decoder/avro/package-info.java b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/decoder/avro/package-info.java new file mode 100644 index 0000000..57bf297 --- /dev/null +++ b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/decoder/avro/package-info.java @@ -0,0 +1,22 @@ +/** + * 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. + */ +/** + * This package contains decoder for SchemaType.AVRO. + */ +package org.apache.pulsar.sql.presto.decoder.avro; \ No newline at end of file diff --git a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/decoder/json/PulsarJsonFieldDecoder.java b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/decoder/json/PulsarJsonFieldDecoder.java new file mode 100644 index 0000000..960d8f4 --- /dev/null +++ b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/decoder/json/PulsarJsonFieldDecoder.java @@ -0,0 +1,438 @@ +/** + * 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.decoder.json; + +import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.base.Preconditions.checkState; +import static io.airlift.slice.Slices.utf8Slice; +import static io.prestosql.decoder.DecoderErrorCode.DECODER_CONVERSION_NOT_SUPPORTED; +import static io.prestosql.spi.type.BigintType.BIGINT; +import static io.prestosql.spi.type.BooleanType.BOOLEAN; +import static io.prestosql.spi.type.DateType.DATE; +import static io.prestosql.spi.type.DoubleType.DOUBLE; +import static io.prestosql.spi.type.IntegerType.INTEGER; +import static io.prestosql.spi.type.RealType.REAL; +import static io.prestosql.spi.type.SmallintType.SMALLINT; +import static io.prestosql.spi.type.TimeType.TIME; +import static io.prestosql.spi.type.TimestampType.TIMESTAMP; +import static io.prestosql.spi.type.TinyintType.TINYINT; +import static io.prestosql.spi.type.Varchars.isVarcharType; +import static io.prestosql.spi.type.Varchars.truncateToLength; +import static java.lang.Double.parseDouble; +import static java.lang.Float.floatToIntBits; +import static java.lang.Float.parseFloat; +import static java.lang.Long.parseLong; +import static java.lang.String.format; +import static java.util.Objects.requireNonNull; + +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.node.ArrayNode; +import com.fasterxml.jackson.databind.node.ObjectNode; +import com.google.common.collect.ImmutableList; +import io.airlift.log.Logger; +import io.airlift.slice.Slice; +import io.prestosql.decoder.DecoderColumnHandle; +import io.prestosql.decoder.FieldValueProvider; +import io.prestosql.decoder.json.JsonFieldDecoder; +import io.prestosql.decoder.json.JsonRowDecoderFactory; +import io.prestosql.spi.PrestoException; +import io.prestosql.spi.block.Block; +import io.prestosql.spi.block.BlockBuilder; +import io.prestosql.spi.type.ArrayType; +import io.prestosql.spi.type.BigintType; +import io.prestosql.spi.type.BooleanType; +import io.prestosql.spi.type.DateType; +import io.prestosql.spi.type.DoubleType; +import io.prestosql.spi.type.IntegerType; +import io.prestosql.spi.type.MapType; +import io.prestosql.spi.type.RealType; +import io.prestosql.spi.type.RowType; +import io.prestosql.spi.type.SmallintType; +import io.prestosql.spi.type.TimeType; +import io.prestosql.spi.type.TimestampType; +import io.prestosql.spi.type.TinyintType; +import io.prestosql.spi.type.Type; +import io.prestosql.spi.type.VarbinaryType; +import io.prestosql.spi.type.VarcharType; +import java.util.Iterator; +import java.util.List; +import java.util.Map; + +import org.apache.commons.lang3.tuple.Pair; + +/** + * Copy from {@link io.prestosql.decoder.json.DefaultJsonFieldDecoder} (presto-record-decoder-345) + * with some pulsar's extensions. + * 1) support {@link io.prestosql.spi.type.ArrayType}. + * 2) support {@link io.prestosql.spi.type.MapType}. + * 3) support {@link io.prestosql.spi.type.RowType}. + * 4) support {@link io.prestosql.spi.type.TimestampType},{@link io.prestosql.spi.type.DateType}, + * {@link io.prestosql.spi.type.TimeType}. + * 5) support {@link io.prestosql.spi.type.RealType}. + */ +public class PulsarJsonFieldDecoder + implements JsonFieldDecoder { + + private final DecoderColumnHandle columnHandle; + private final long minValue; + private final long maxValue; + + public PulsarJsonFieldDecoder(DecoderColumnHandle columnHandle) { + this.columnHandle = requireNonNull(columnHandle, "columnHandle is null"); + if (!isSupportedType(columnHandle.getType())) { + JsonRowDecoderFactory.throwUnsupportedColumnType(columnHandle); + } + Pair range = getNumRangeByType(columnHandle.getType()); + minValue = range.getKey(); + maxValue = range.getValue(); + + } + + private static Pair getNumRangeByType(Type type) { + if (type == TINYINT) { + return Pair.of((long) Byte.MIN_VALUE, (long) Byte.MAX_VALUE); + } else if (type == SMALLINT) { + return Pair.of((long) Short.MIN_VALUE, (long) Short.MAX_VALUE); + } else if (type == INTEGER) { + return Pair.of((long) Integer.MIN_VALUE, (long) Integer.MAX_VALUE); + } else if (type == BIGINT) { + return Pair.of((long) Long.MIN_VALUE, (long) Long.MAX_VALUE); + } else { + // those values will not be used if column type is not one of mentioned above + return Pair.of(Long.MIN_VALUE, Long.MAX_VALUE); + } + } + + private boolean isSupportedType(Type type) { + if (isVarcharType(type)) { + return true; + } + if (ImmutableList.of( + BIGINT, + INTEGER, + SMALLINT, + TINYINT, + BOOLEAN, + DOUBLE, + TIMESTAMP, + DATE, + TIME, + REAL + ).contains(type)) { + return true; + } + + if (type instanceof ArrayType) { + checkArgument(type.getTypeParameters().size() == 1, "expecting exactly one type parameter for array"); + return isSupportedType(type.getTypeParameters().get(0)); + } + if (type instanceof MapType) { + List typeParameters = type.getTypeParameters(); + checkArgument(typeParameters.size() == 2, "expecting exactly two type parameters for map"); + return isSupportedType(type.getTypeParameters().get(0)) && isSupportedType(type.getTypeParameters().get(1)); + } + + if (type instanceof RowType) { + for (Type fieldType : type.getTypeParameters()) { + if (!isSupportedType(fieldType)) { + return false; + } + } + return true; + } + + return false; + } + + @Override + public FieldValueProvider decode(JsonNode value) { + return new JsonValueProvider(value, columnHandle, minValue, maxValue); + } + + /** + * JsonValueProvider. + */ + public static class JsonValueProvider + extends FieldValueProvider { + private final JsonNode value; + private final DecoderColumnHandle columnHandle; + private final long minValue; + private final long maxValue; + + public JsonValueProvider(JsonNode value, DecoderColumnHandle columnHandle, long minValue, long maxValue) { + this.value = value; + this.columnHandle = columnHandle; + this.minValue = minValue; + this.maxValue = maxValue; + } + + @Override + public final boolean isNull() { + return value.isMissingNode() || value.isNull(); + } + + @Override + public boolean getBoolean() { + return getBoolean(value, columnHandle.getType(), columnHandle.getName()); + } + + @Override + public long getLong() { + return getLong(value, columnHandle.getType(), columnHandle.getName(), minValue, maxValue); + } + + @Override + public double getDouble() { + return getDouble(value, columnHandle.getType(), columnHandle.getName()); + } + + @Override + public Slice getSlice() { + return getSlice(value, columnHandle.getType(), columnHandle.getName()); + } + + @Override + public Block getBlock() { + return serializeObject(null, value, columnHandle.getType(), columnHandle.getName()); + } + + + public static boolean getBoolean(JsonNode value, Type type, String columnName) { + if (value.isValueNode()) { + return value.asBoolean(); + } + throw new PrestoException( + DECODER_CONVERSION_NOT_SUPPORTED, + format("could not parse non-value node as '%s' for column '%s'", type, columnName)); + } + + public static long getLong(JsonNode value, Type type, String columnName, long minValue, long maxValue) { + try { + if (type instanceof RealType) { + return floatToIntBits((Float) parseFloat(value.asText())); + } + + long longValue; + if (value.isIntegralNumber() && !value.isBigInteger()) { + longValue = value.longValue(); + if (longValue >= minValue && longValue <= maxValue) { + return longValue; + } + } else if (value.isValueNode()) { + longValue = parseLong(value.asText()); + if (longValue >= minValue && longValue <= maxValue) { + return longValue; + } + } + } catch (NumberFormatException ignore) { + // ignore + } + throw new PrestoException( + DECODER_CONVERSION_NOT_SUPPORTED, + format("could not parse value '%s' as '%s' for column '%s'", value.asText(), type, columnName)); + } + + public static double getDouble(JsonNode value, Type type, String columnName) { + try { + if (value.isNumber()) { + return value.doubleValue(); + } + if (value.isValueNode()) { + return parseDouble(value.asText()); + } + } catch (NumberFormatException ignore) { + // ignore + } + throw new PrestoException( + DECODER_CONVERSION_NOT_SUPPORTED, + format("could not parse value '%s' as '%s' for column '%s'", value.asText(), type, columnName)); + + } + + private static Slice getSlice(JsonNode value, Type type, String columnName) { + String textValue = value.isValueNode() ? value.asText() : value.toString(); + Slice slice = utf8Slice(textValue); + if (isVarcharType(type)) { + slice = truncateToLength(slice, type); + } + return slice; + } + + private Block serializeObject(BlockBuilder builder, Object value, Type type, String columnName) { + if (type instanceof ArrayType) { + return serializeList(builder, value, type, columnName); + } + if (type instanceof MapType) { + return serializeMap(builder, value, type, columnName); + } + if (type instanceof RowType) { + return serializeRow(builder, value, type, columnName); + } + serializePrimitive(builder, value, type, columnName); + return null; + } + + private Block serializeList(BlockBuilder parentBlockBuilder, Object value, Type type, String columnName) { + if (value == null) { + checkState(parentBlockBuilder != null, "parentBlockBuilder is null"); + parentBlockBuilder.appendNull(); + return null; + } + + checkState(value instanceof ArrayNode, "Json array node must is ArrayNode type"); + + Iterator jsonNodeIterator = ((ArrayNode) value).elements(); + + List typeParameters = type.getTypeParameters(); + Type elementType = typeParameters.get(0); + + BlockBuilder blockBuilder = elementType.createBlockBuilder(null, ((ArrayNode) value).size()); + + while (jsonNodeIterator.hasNext()) { + Object element = jsonNodeIterator.next(); + serializeObject(blockBuilder, element, elementType, columnName); + } + + if (parentBlockBuilder != null) { + type.writeObject(parentBlockBuilder, blockBuilder.build()); + return null; + } + return blockBuilder.build(); + } + + private void serializePrimitive(BlockBuilder blockBuilder, Object node, Type type, String columnName) { + requireNonNull(blockBuilder, "parent blockBuilder is null"); + + JsonNode value; + if (node == null) { + blockBuilder.appendNull(); + return; + } + + if (node instanceof JsonNode) { + value = (JsonNode) node; + } else { + throw new PrestoException(DECODER_CONVERSION_NOT_SUPPORTED, + format("primitive object of '%s' as '%s' for column '%s' cann't convert to JsonNode", + node.getClass(), type, columnName)); + } + + if (type instanceof BooleanType) { + type.writeBoolean(blockBuilder, getBoolean(value, type, columnName)); + return; + } + + if (type instanceof RealType || type instanceof BigintType + || type instanceof IntegerType || type instanceof SmallintType + || type instanceof TinyintType || type instanceof TimestampType + || type instanceof TimeType || type instanceof DateType) { + Pair numRange = getNumRangeByType(type); + type.writeLong(blockBuilder, getLong(value, type, columnName, numRange.getKey(), numRange.getValue())); + return; + } + + if (type instanceof DoubleType) { + type.writeDouble(blockBuilder, getDouble(value, type, columnName)); + return; + } + + if (type instanceof VarcharType || type instanceof VarbinaryType) { + type.writeSlice(blockBuilder, getSlice(value, type, columnName)); + return; + } + + throw new PrestoException(DECODER_CONVERSION_NOT_SUPPORTED, + format("cannot decode object of '%s' as '%s' for column '%s'", value.getClass(), type, columnName)); + } + + private Block serializeMap(BlockBuilder parentBlockBuilder, Object value, Type type, String columnName) { + if (value == null) { + checkState(parentBlockBuilder != null, "parentBlockBuilder is null"); + parentBlockBuilder.appendNull(); + return null; + } + checkState(value instanceof ObjectNode, "Json map node must is ObjectNode type"); + + List typeParameters = type.getTypeParameters(); + Type keyType = typeParameters.get(0); + Type valueType = typeParameters.get(1); + + BlockBuilder blockBuilder; + if (parentBlockBuilder != null) { + blockBuilder = parentBlockBuilder; + } else { + blockBuilder = type.createBlockBuilder(null, 1); + } + + BlockBuilder entryBuilder = blockBuilder.beginBlockEntry(); + + Iterator> fields = ((ObjectNode) value).fields(); + while (fields.hasNext()) { + Map.Entry entry = fields.next(); + if (entry.getKey() != null) { + keyType.writeSlice(entryBuilder, truncateToLength(utf8Slice(entry.getKey().toString()), keyType)); + serializeObject(entryBuilder, entry.getValue(), valueType, columnName); + } + } + + blockBuilder.closeEntry(); + + if (parentBlockBuilder == null) { + return blockBuilder.getObject(0, Block.class); + } + return null; + } + + + private Block serializeRow(BlockBuilder parentBlockBuilder, Object value, Type type, String columnName) { + if (value == null) { + checkState(parentBlockBuilder != null, "parent block builder is null"); + parentBlockBuilder.appendNull(); + return null; + } + + BlockBuilder blockBuilder; + if (parentBlockBuilder != null) { + blockBuilder = parentBlockBuilder; + } else { + blockBuilder = type.createBlockBuilder(null, 1); + } + BlockBuilder singleRowBuilder = blockBuilder.beginBlockEntry(); + + List fields = ((RowType) type).getFields(); + + checkState(value instanceof ObjectNode, "Json row node must be ObjectNode type"); + + for (RowType.Field field : fields) { + checkState(field.getName().isPresent(), "field name not found"); + serializeObject(singleRowBuilder, ((ObjectNode) value).get(field.getName().get()), + field.getType(), columnName); + } + blockBuilder.closeEntry(); + if (parentBlockBuilder == null) { + return blockBuilder.getObject(0, Block.class); + } + return null; + } + + } + + private static final Logger log = Logger.get(PulsarJsonFieldDecoder.class); + +} diff --git a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/decoder/json/PulsarJsonRowDecoder.java b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/decoder/json/PulsarJsonRowDecoder.java new file mode 100644 index 0000000..638a723 --- /dev/null +++ b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/decoder/json/PulsarJsonRowDecoder.java @@ -0,0 +1,89 @@ +/** + * 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.decoder.json; + +import static com.google.common.base.Preconditions.checkState; +import static com.google.common.collect.ImmutableMap.toImmutableMap; +import static java.util.Objects.requireNonNull; +import static java.util.function.Function.identity; + +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.node.MissingNode; +import com.google.common.base.Splitter; +import io.airlift.log.Logger; +import io.netty.buffer.ByteBuf; +import io.prestosql.decoder.DecoderColumnHandle; +import io.prestosql.decoder.FieldValueProvider; +import io.prestosql.decoder.json.JsonFieldDecoder; +import java.util.HashMap; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import org.apache.pulsar.client.impl.schema.generic.GenericJsonRecord; +import org.apache.pulsar.client.impl.schema.generic.GenericJsonSchema; +import org.apache.pulsar.sql.presto.PulsarRowDecoder; + +/** + * Json PulsarRowDecoder. + */ +public class PulsarJsonRowDecoder implements PulsarRowDecoder { + + private final Map fieldDecoders; + + private final GenericJsonSchema genericJsonSchema; + + public PulsarJsonRowDecoder(GenericJsonSchema genericJsonSchema, Set columns) { + this.genericJsonSchema = requireNonNull(genericJsonSchema, "genericJsonSchema is null"); + this.fieldDecoders = columns.stream().collect(toImmutableMap(identity(), PulsarJsonFieldDecoder::new)); + } + + private static JsonNode locateNode(JsonNode tree, DecoderColumnHandle columnHandle) { + String mapping = columnHandle.getMapping(); + checkState(mapping != null, "No mapping for %s", columnHandle.getName()); + JsonNode currentNode = tree; + for (String pathElement : Splitter.on('/').omitEmptyStrings().split(mapping)) { + if (!currentNode.has(pathElement)) { + return MissingNode.getInstance(); + } + currentNode = currentNode.path(pathElement); + } + return currentNode; + } + + /** + * decode ByteBuf by {@link org.apache.pulsar.client.api.schema.GenericSchema}. + * @param byteBuf + * @return + */ + @Override + public Optional> decodeRow(ByteBuf byteBuf) { + GenericJsonRecord record = (GenericJsonRecord) genericJsonSchema.decode(byteBuf); + JsonNode tree = record.getJsonNode(); + Map decodedRow = new HashMap<>(); + for (Map.Entry entry : fieldDecoders.entrySet()) { + DecoderColumnHandle columnHandle = entry.getKey(); + JsonFieldDecoder decoder = entry.getValue(); + JsonNode node = locateNode(tree, columnHandle); + decodedRow.put(columnHandle, decoder.decode(node)); + } + return Optional.of(decodedRow); + } + + private static final Logger log = Logger.get(PulsarJsonRowDecoderFactory.class); +} diff --git a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/decoder/json/PulsarJsonRowDecoderFactory.java b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/decoder/json/PulsarJsonRowDecoderFactory.java new file mode 100644 index 0000000..10a500b --- /dev/null +++ b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/decoder/json/PulsarJsonRowDecoderFactory.java @@ -0,0 +1,186 @@ +/** + * 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.decoder.json; + +import static com.google.common.collect.ImmutableList.toImmutableList; +import static io.prestosql.spi.StandardErrorCode.NOT_SUPPORTED; +import static io.prestosql.spi.type.DateType.DATE; +import static io.prestosql.spi.type.TimeType.TIME; +import static io.prestosql.spi.type.VarcharType.createUnboundedVarcharType; +import static java.lang.String.format; +import static java.util.stream.Collectors.toList; + +import com.google.common.collect.ImmutableList; +import io.airlift.log.Logger; +import io.prestosql.decoder.DecoderColumnHandle; +import io.prestosql.spi.PrestoException; +import io.prestosql.spi.connector.ColumnMetadata; +import io.prestosql.spi.type.ArrayType; +import io.prestosql.spi.type.BigintType; +import io.prestosql.spi.type.BooleanType; +import io.prestosql.spi.type.DoubleType; +import io.prestosql.spi.type.IntegerType; +import io.prestosql.spi.type.RealType; +import io.prestosql.spi.type.RowType; +import io.prestosql.spi.type.StandardTypes; +import io.prestosql.spi.type.TimestampType; +import io.prestosql.spi.type.Type; +import io.prestosql.spi.type.TypeManager; +import io.prestosql.spi.type.TypeSignature; +import io.prestosql.spi.type.TypeSignatureParameter; +import io.prestosql.spi.type.VarbinaryType; +import io.prestosql.spi.type.VarcharType; +import java.util.List; +import java.util.Optional; +import java.util.Set; +import org.apache.avro.LogicalType; +import org.apache.avro.LogicalTypes; +import org.apache.avro.Schema; +import org.apache.avro.SchemaParseException; +import org.apache.commons.lang3.StringUtils; +import org.apache.pulsar.client.impl.schema.generic.GenericJsonSchema; +import org.apache.pulsar.common.naming.TopicName; +import org.apache.pulsar.common.schema.SchemaInfo; +import org.apache.pulsar.sql.presto.PulsarColumnHandle; +import org.apache.pulsar.sql.presto.PulsarColumnMetadata; +import org.apache.pulsar.sql.presto.PulsarRowDecoderFactory; + +/** + * PulsarRowDecoderFactory for {@link org.apache.pulsar.common.schema.SchemaType#JSON}. + */ +public class PulsarJsonRowDecoderFactory implements PulsarRowDecoderFactory { + + private TypeManager typeManager; + + public PulsarJsonRowDecoderFactory(TypeManager typeManager) { + this.typeManager = typeManager; + } + + @Override + public PulsarJsonRowDecoder createRowDecoder(TopicName topicName, SchemaInfo schemaInfo, + Set columns) { + return new PulsarJsonRowDecoder((GenericJsonSchema) GenericJsonSchema.of(schemaInfo), columns); + } + + @Override + public List extractColumnMetadata(TopicName topicName, SchemaInfo schemaInfo, + PulsarColumnHandle.HandleKeyValueType handleKeyValueType) { + List columnMetadata; + String schemaJson = new String(schemaInfo.getSchema()); + if (StringUtils.isBlank(schemaJson)) { + throw new PrestoException(NOT_SUPPORTED, "Topic " + + topicName.toString() + " does not have a valid schema"); + } + + Schema schema; + try { + schema = GenericJsonSchema.of(schemaInfo).getAvroSchema(); + } catch (SchemaParseException ex) { + throw new PrestoException(NOT_SUPPORTED, "Topic " + + topicName.toString() + " does not have a valid schema"); + } + + try { + columnMetadata = schema.getFields().stream() + .map(field -> + new PulsarColumnMetadata(PulsarColumnMetadata.getColumnName(handleKeyValueType, field.name()), parseJsonPrestoType(field.name(), field.schema()), + field.schema().toString(), null, false, false, + handleKeyValueType, new PulsarColumnMetadata.DecoderExtraInfo( + field.name(), null, null)) + + ).collect(toList()); + } catch (StackOverflowError e) { + log.warn(e, "Topic " + + topicName.toString() + " extractColumnMetadata failed."); + throw new PrestoException(NOT_SUPPORTED, "Topic " + + topicName.toString() + " schema may contains cyclic definitions.", e); + } + return columnMetadata; + } + + + private Type parseJsonPrestoType(String fieldname, Schema schema) { + Schema.Type type = schema.getType(); + LogicalType logicalType = schema.getLogicalType(); + switch (type) { + case STRING: + case ENUM: + return createUnboundedVarcharType(); + case NULL: + throw new UnsupportedOperationException(format( + "field '%s' NULL type code should not be reached ," + + "please check the schema or report the bug.", fieldname)); + case FIXED: + case BYTES: + return VarbinaryType.VARBINARY; + case INT: + if (logicalType == LogicalTypes.timeMillis()) { + return TIME; + } else if (logicalType == LogicalTypes.date()) { + return DATE; + } + return IntegerType.INTEGER; + case LONG: + if (logicalType == LogicalTypes.timestampMillis()) { + return TimestampType.TIMESTAMP; + } + return BigintType.BIGINT; + case FLOAT: + return RealType.REAL; + case DOUBLE: + return DoubleType.DOUBLE; + case BOOLEAN: + return BooleanType.BOOLEAN; + case ARRAY: + return new ArrayType(parseJsonPrestoType(fieldname, schema.getElementType())); + case MAP: + //The key for an avro map must be string. + TypeSignature valueType = parseJsonPrestoType(fieldname, schema.getValueType()).getTypeSignature(); + return typeManager.getParameterizedType(StandardTypes.MAP, ImmutableList.of(TypeSignatureParameter. + typeParameter(VarcharType.VARCHAR.getTypeSignature()), + TypeSignatureParameter.typeParameter(valueType))); + case RECORD: + if (schema.getFields().size() > 0) { + return RowType.from(schema.getFields().stream() + .map(field -> new RowType.Field(Optional.of(field.name()), + parseJsonPrestoType(field.name(), field.schema()))) + .collect(toImmutableList())); + } else { + throw new UnsupportedOperationException(format( + "field '%s' of record type has no fields, " + + "please check schema definition. ", fieldname)); + } + case UNION: + for (Schema nestType : schema.getTypes()) { + if (nestType.getType() != Schema.Type.NULL) { + return parseJsonPrestoType(fieldname, nestType); + } + } + throw new UnsupportedOperationException(format( + "field '%s' of UNION type must contains not NULL type.", fieldname)); + default: + throw new UnsupportedOperationException(format( + "Can't convert from schema type '%s' (%s) to presto type.", + schema.getType(), schema.getFullName())); + } + } + + private static final Logger log = Logger.get(PulsarJsonRowDecoderFactory.class); + +} diff --git a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/decoder/json/package-info.java b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/decoder/json/package-info.java new file mode 100644 index 0000000..a8d8b67 --- /dev/null +++ b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/decoder/json/package-info.java @@ -0,0 +1,22 @@ +/** + * 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. + */ +/** + * This package contains decoder for SchemaType.JSON. + */ +package org.apache.pulsar.sql.presto.decoder.json; \ No newline at end of file diff --git a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/decoder/primitive/PulsarPrimitiveRowDecoder.java b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/decoder/primitive/PulsarPrimitiveRowDecoder.java new file mode 100644 index 0000000..f2da259 --- /dev/null +++ b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/decoder/primitive/PulsarPrimitiveRowDecoder.java @@ -0,0 +1,101 @@ +/** + * 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.decoder.primitive; + +import static io.prestosql.decoder.FieldValueProviders.booleanValueProvider; +import static io.prestosql.decoder.FieldValueProviders.bytesValueProvider; +import static io.prestosql.decoder.FieldValueProviders.longValueProvider; +import static org.apache.pulsar.sql.presto.PulsarFieldValueProviders.doubleValueProvider; + +import io.netty.buffer.ByteBuf; +import io.prestosql.decoder.DecoderColumnHandle; +import io.prestosql.decoder.FieldValueProvider; +import io.prestosql.decoder.FieldValueProviders; +import io.prestosql.spi.type.BigintType; +import io.prestosql.spi.type.BooleanType; +import io.prestosql.spi.type.DateType; +import io.prestosql.spi.type.DoubleType; +import io.prestosql.spi.type.IntegerType; +import io.prestosql.spi.type.RealType; +import io.prestosql.spi.type.SmallintType; +import io.prestosql.spi.type.TimeType; +import io.prestosql.spi.type.TimestampType; +import io.prestosql.spi.type.TinyintType; +import io.prestosql.spi.type.Type; +import io.prestosql.spi.type.VarbinaryType; +import io.prestosql.spi.type.VarcharType; + +import java.sql.Time; +import java.sql.Timestamp; +import java.util.Date; +import java.util.HashMap; +import java.util.Map; +import java.util.Optional; + +import org.apache.pulsar.client.impl.schema.AbstractSchema; +import org.apache.pulsar.sql.presto.PulsarRowDecoder; + +/** + * Primitive Schema PulsarRowDecoder. + */ +public class PulsarPrimitiveRowDecoder implements PulsarRowDecoder { + + private final DecoderColumnHandle columnHandle; + private AbstractSchema schema; + + public PulsarPrimitiveRowDecoder(AbstractSchema schema, DecoderColumnHandle column) { + this.columnHandle = column; + this.schema = schema; + } + + @Override + public Optional> decodeRow(ByteBuf byteBuf) { + Object value = schema.decode(byteBuf); + Map primitiveColumn = new HashMap<>(); + if (value == null) { + primitiveColumn.put(columnHandle, FieldValueProviders.nullValueProvider()); + } else { + Type type = columnHandle.getType(); + if (type instanceof BooleanType) { + primitiveColumn.put(columnHandle, booleanValueProvider(Boolean.valueOf((Boolean) value))); + } else if (type instanceof TinyintType || type instanceof SmallintType || type instanceof IntegerType + || type instanceof BigintType) { + primitiveColumn.put(columnHandle, longValueProvider(Long.valueOf(value.toString()))); + } else if (type instanceof DoubleType) { + primitiveColumn.put(columnHandle, doubleValueProvider(Double.valueOf(value.toString()))); + } else if (type instanceof RealType) { + primitiveColumn.put(columnHandle, longValueProvider( + Float.floatToIntBits((Float.valueOf(value.toString()))))); + } else if (type instanceof VarbinaryType) { + primitiveColumn.put(columnHandle, bytesValueProvider((byte[]) value)); + } else if (type instanceof VarcharType) { + primitiveColumn.put(columnHandle, bytesValueProvider(value.toString().getBytes())); + } else if (type instanceof DateType) { + primitiveColumn.put(columnHandle, longValueProvider(((Date) value).getTime())); + } else if (type instanceof TimeType) { + primitiveColumn.put(columnHandle, longValueProvider(((Time) value).getTime())); + } else if (type instanceof TimestampType) { + primitiveColumn.put(columnHandle, longValueProvider(((Timestamp) value).getTime())); + } else { + primitiveColumn.put(columnHandle, bytesValueProvider(value.toString().getBytes())); + } + } + return Optional.of(primitiveColumn); + } +} diff --git a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/decoder/primitive/PulsarPrimitiveRowDecoderFactory.java b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/decoder/primitive/PulsarPrimitiveRowDecoderFactory.java new file mode 100644 index 0000000..9afddfe --- /dev/null +++ b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/decoder/primitive/PulsarPrimitiveRowDecoderFactory.java @@ -0,0 +1,115 @@ +/** + * 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.decoder.primitive; + +import io.airlift.log.Logger; +import io.prestosql.decoder.DecoderColumnHandle; +import io.prestosql.spi.connector.ColumnMetadata; +import io.prestosql.spi.type.BigintType; +import io.prestosql.spi.type.BooleanType; +import io.prestosql.spi.type.DateType; +import io.prestosql.spi.type.DoubleType; +import io.prestosql.spi.type.IntegerType; +import io.prestosql.spi.type.RealType; +import io.prestosql.spi.type.SmallintType; +import io.prestosql.spi.type.TimeType; +import io.prestosql.spi.type.TimestampType; +import io.prestosql.spi.type.TinyintType; +import io.prestosql.spi.type.Type; +import io.prestosql.spi.type.VarbinaryType; +import io.prestosql.spi.type.VarcharType; +import java.util.Arrays; +import java.util.List; +import java.util.Set; +import org.apache.pulsar.client.impl.schema.AbstractSchema; +import org.apache.pulsar.client.impl.schema.AutoConsumeSchema; +import org.apache.pulsar.common.naming.TopicName; +import org.apache.pulsar.common.schema.SchemaInfo; +import org.apache.pulsar.common.schema.SchemaType; +import org.apache.pulsar.sql.presto.PulsarColumnHandle; +import org.apache.pulsar.sql.presto.PulsarColumnMetadata; +import org.apache.pulsar.sql.presto.PulsarRowDecoder; +import org.apache.pulsar.sql.presto.PulsarRowDecoderFactory; + +/** + * Primitive Schema PulsarRowDecoderFactory. + */ +public class PulsarPrimitiveRowDecoderFactory implements PulsarRowDecoderFactory { + + private static final Logger log = Logger.get(PulsarPrimitiveRowDecoderFactory.class); + + public static final String PRIMITIVE_COLUMN_NAME = "__value__"; + + @Override + public PulsarRowDecoder createRowDecoder(TopicName topicName, SchemaInfo schemaInfo, + Set columns) { + if (columns.size() == 1) { + return new PulsarPrimitiveRowDecoder((AbstractSchema) AutoConsumeSchema.getSchema(schemaInfo), + columns.iterator().next()); + } else { + throw new RuntimeException("Primitive type must has only one ColumnHandle."); + } + } + + @Override + public List extractColumnMetadata(TopicName topicName, SchemaInfo schemaInfo, + PulsarColumnHandle.HandleKeyValueType handleKeyValueType) { + ColumnMetadata valueColumn = new PulsarColumnMetadata( + PulsarColumnMetadata.getColumnName(handleKeyValueType, PRIMITIVE_COLUMN_NAME), + parsePrimitivePrestoType(PRIMITIVE_COLUMN_NAME, schemaInfo.getType()), + "The value of the message with primitive type schema", null, false, false, + handleKeyValueType, new PulsarColumnMetadata.DecoderExtraInfo(PRIMITIVE_COLUMN_NAME, + null, null)); + return Arrays.asList(valueColumn); + } + + private Type parsePrimitivePrestoType(String fieldName, SchemaType pulsarType) { + switch (pulsarType) { + case BOOLEAN: + return BooleanType.BOOLEAN; + case INT8: + return TinyintType.TINYINT; + case INT16: + return SmallintType.SMALLINT; + case INT32: + return IntegerType.INTEGER; + case INT64: + return BigintType.BIGINT; + case FLOAT: + return RealType.REAL; + case DOUBLE: + return DoubleType.DOUBLE; + case NONE: + case BYTES: + return VarbinaryType.VARBINARY; + case STRING: + return VarcharType.VARCHAR; + case DATE: + return DateType.DATE; + case TIME: + return TimeType.TIME; + case TIMESTAMP: + return TimestampType.TIMESTAMP; + default: + log.error("Can't convert type: %s for %s", pulsarType, fieldName); + return null; + } + + } +} diff --git a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/decoder/primitive/package-info.java b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/decoder/primitive/package-info.java new file mode 100644 index 0000000..7ceecfb --- /dev/null +++ b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/decoder/primitive/package-info.java @@ -0,0 +1,22 @@ +/** + * 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. + */ +/** + * This package contains decoder for SchemaType of SchemaType.isPrimitive() return true. + */ +package org.apache.pulsar.sql.presto.decoder.primitive; \ No newline at end of file diff --git a/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/TestAvroSchemaHandler.java b/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/TestAvroSchemaHandler.java deleted file mode 100644 index 4dbda98..0000000 --- a/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/TestAvroSchemaHandler.java +++ /dev/null @@ -1,127 +0,0 @@ -/** - * 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.netty.buffer.ByteBufAllocator; -import io.prestosql.spi.type.BigintType; -import lombok.Data; -import lombok.extern.slf4j.Slf4j; -import org.apache.avro.Schema; -import org.apache.avro.io.BinaryEncoder; -import org.apache.avro.io.EncoderFactory; -import org.apache.avro.reflect.ReflectData; -import org.apache.avro.reflect.ReflectDatumWriter; -import org.apache.pulsar.client.api.schema.SchemaDefinition; -import org.apache.pulsar.client.impl.schema.generic.GenericAvroRecord; -import org.apache.pulsar.common.api.raw.RawMessage; -import org.apache.pulsar.common.schema.SchemaType; -import org.testng.Assert; -import org.testng.annotations.Test; - -import java.io.ByteArrayOutputStream; -import java.io.IOException; -import java.util.ArrayList; -import java.util.List; - -import static java.util.concurrent.CompletableFuture.completedFuture; -import static org.apache.pulsar.client.impl.schema.util.SchemaUtil.parseSchemaInfo; -import static org.mockito.Matchers.any; -import static org.mockito.Mockito.when; - -import static org.mockito.Mockito.mock; - -@Slf4j -public class TestAvroSchemaHandler { - - @Data - public static class Foo1 { - String field1; - - Bar bar; - } - @Data - public static class Foo2 { - String field1; - - String field2; - - Bar bar; - } - - @Data static class Bar { - String field1; - - String field2; - } - - @Test - public void testAvroSchemaHandler() throws IOException { - List columnHandles = new ArrayList(); - RawMessage message = mock(RawMessage.class); - Schema schema1 = ReflectData.AllowNull.get().getSchema(Foo1.class); - PulsarSqlSchemaInfoProvider pulsarSqlSchemaInfoProvider = mock(PulsarSqlSchemaInfoProvider.class); - AvroSchemaHandler avroSchemaHandler = new AvroSchemaHandler(pulsarSqlSchemaInfoProvider, - parseSchemaInfo(SchemaDefinition.builder().withPojo(Foo2.class).build(), SchemaType.AVRO), columnHandles); - byte[] schemaVersion = new byte[8]; - for (int i = 0 ; i<8; i++) { - schemaVersion[i] = 0; - } - ReflectDatumWriter writer; - BinaryEncoder encoder = null; - ByteArrayOutputStream byteArrayOutputStream; - byteArrayOutputStream = new ByteArrayOutputStream(); - encoder = EncoderFactory.get().binaryEncoder(byteArrayOutputStream, encoder); - writer = new ReflectDatumWriter<>(schema1); - Foo1 foo1 = new Foo1(); - foo1.setField1("value1"); - foo1.setBar(new Bar()); - foo1.getBar().setField1("value1"); - writer.write(foo1, encoder); - encoder.flush(); - when(message.getSchemaVersion()).thenReturn(schemaVersion); - byte[] bytes =byteArrayOutputStream.toByteArray(); - - when(message.getData()).thenReturn(ByteBufAllocator.DEFAULT - .buffer(bytes.length, bytes.length).writeBytes(byteArrayOutputStream.toByteArray())); - when(pulsarSqlSchemaInfoProvider.getSchemaByVersion(any())) - .thenReturn(completedFuture(parseSchemaInfo(SchemaDefinition.builder() - .withPojo(Foo1.class).build(), SchemaType.AVRO))); - - Object object = ((GenericAvroRecord)avroSchemaHandler.deserialize(message.getData(), - message.getSchemaVersion())).getField("field1"); - Assert.assertEquals(foo1.field1, (String)object); - String[] fields = new String[2]; - fields[0] = "bar"; - fields[1] = "field1"; - PulsarColumnHandle pulsarColumnHandle = new PulsarColumnHandle("test", - "bar.field1", - BigintType.BIGINT, - true, - true, - fields, - new Integer[5], - null); - columnHandles.add(pulsarColumnHandle); - when(message.getData()).thenReturn(ByteBufAllocator.DEFAULT - .buffer(bytes.length, bytes.length).writeBytes(byteArrayOutputStream.toByteArray())); - object = avroSchemaHandler.extractField(0, avroSchemaHandler.deserialize(message.getData(), - message.getSchemaVersion())); - Assert.assertEquals(foo1.bar.field1, (String)object); - } -} diff --git a/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/TestPulsarConnector.java b/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/TestPulsarConnector.java index 38ba863..5dc2afb 100644 --- a/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/TestPulsarConnector.java +++ b/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/TestPulsarConnector.java @@ -21,53 +21,38 @@ import com.fasterxml.jackson.databind.ObjectMapper; import io.airlift.log.Logger; import io.netty.buffer.ByteBuf; +import io.prestosql.spi.connector.ColumnMetadata; +import io.prestosql.spi.connector.ConnectorContext; import io.prestosql.spi.predicate.TupleDomain; -import io.prestosql.spi.type.BigintType; -import io.prestosql.spi.type.BooleanType; -import io.prestosql.spi.type.DoubleType; -import io.prestosql.spi.type.IntegerType; -import io.prestosql.spi.type.RealType; -import io.prestosql.spi.type.Type; -import io.prestosql.spi.type.VarcharType; -import org.apache.bookkeeper.mledger.AsyncCallbacks; -import org.apache.bookkeeper.mledger.Entry; -import org.apache.bookkeeper.mledger.ManagedLedgerConfig; -import org.apache.bookkeeper.mledger.ManagedLedgerFactory; -import org.apache.bookkeeper.mledger.Position; -import org.apache.bookkeeper.mledger.ReadOnlyCursor; +import io.prestosql.testing.TestingConnectorContext; +import org.apache.bookkeeper.mledger.*; import org.apache.bookkeeper.mledger.impl.EntryImpl; import org.apache.bookkeeper.mledger.impl.PositionImpl; import org.apache.bookkeeper.mledger.impl.ReadOnlyCursorImpl; import org.apache.bookkeeper.mledger.proto.MLDataFormats; +import org.apache.bookkeeper.stats.NullStatsProvider; import org.apache.commons.lang3.StringUtils; -import org.apache.pulsar.client.admin.Namespaces; -import org.apache.pulsar.client.admin.PulsarAdmin; -import org.apache.pulsar.client.admin.PulsarAdminException; -import org.apache.pulsar.client.admin.Schemas; -import org.apache.pulsar.client.admin.Tenants; -import org.apache.pulsar.client.admin.Topics; +import org.apache.pulsar.client.admin.*; import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.client.api.schema.SchemaDefinition; import org.apache.pulsar.client.impl.schema.AvroSchema; import org.apache.pulsar.client.impl.schema.JSONSchema; -import org.apache.pulsar.common.protocol.Commands; import org.apache.pulsar.common.api.proto.MessageMetadata; import org.apache.pulsar.common.naming.NamespaceName; import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.partition.PartitionedTopicMetadata; +import org.apache.pulsar.common.protocol.Commands; import org.apache.pulsar.common.schema.SchemaInfo; import org.apache.pulsar.common.schema.SchemaType; -import javax.ws.rs.ClientErrorException; -import javax.ws.rs.core.Response; -import org.apache.bookkeeper.stats.NullStatsProvider; import org.mockito.Mockito; import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; import org.testng.annotations.AfterMethod; import org.testng.annotations.BeforeMethod; import org.testng.annotations.DataProvider; -import org.testng.annotations.Test; +import javax.ws.rs.ClientErrorException; +import javax.ws.rs.core.Response; import java.time.LocalDate; import java.time.LocalTime; import java.time.ZoneId; @@ -81,9 +66,6 @@ import java.util.function.Function; import java.util.stream.Collectors; -import static io.prestosql.spi.type.DateType.DATE; -import static io.prestosql.spi.type.TimeType.TIME; -import static io.prestosql.spi.type.TimestampType.TIMESTAMP; import static org.apache.pulsar.common.protocol.Commands.serializeMetadataAndPayload; import static org.mockito.Mockito.any; import static org.mockito.Mockito.anyInt; @@ -93,6 +75,7 @@ import static org.mockito.Mockito.mock; import static org.mockito.Mockito.spy; import static org.mockito.Mockito.when; +import static org.testng.Assert.assertNotNull; public abstract class TestPulsarConnector { @@ -110,6 +93,8 @@ public abstract class TestPulsarConnector { protected Map pulsarRecordCursors = new HashMap<>(); + protected static PulsarDispatchingRowDecoderFactory dispatchingRowDecoderFactory; + protected final static PulsarConnectorId pulsarConnectorId = new PulsarConnectorId("test-connector"); protected static List topicNames; @@ -120,6 +105,8 @@ public abstract class TestPulsarConnector { private final static ObjectMapper objectMapper = new ObjectMapper(); + protected static List fooFieldNames = new ArrayList<>(); + protected static final NamespaceName NAMESPACE_NAME_1 = NamespaceName.get("tenant-1", "ns-1"); protected static final NamespaceName NAMESPACE_NAME_2 = NamespaceName.get("tenant-1", "ns-2"); protected static final NamespaceName NAMESPACE_NAME_3 = NamespaceName.get("tenant-2", "ns-1"); @@ -150,10 +137,6 @@ public abstract class TestPulsarConnector { public static class Foo { - public static class Bar { - public int field1; - } - public enum TestEnum { TEST_ENUM_1, TEST_ENUM_2, @@ -179,28 +162,13 @@ public enum TestEnum { public static class Bar { public Integer field1; public String field2; - public Boo test; public float field3; - public Boo test2; } - public static class Boo { - public Double field4; - public Boolean field5; - public long field6; - // for test cyclic definitions - public Foo foo; - public Boo boo; - public Bar bar; - // different namespace with same classname should work though - public Foo.Bar foobar; - } - protected static Map fooTypes; - protected static List fooColumnHandles; + protected static Map> topicsToColumnHandles = new HashMap<>(); + protected static Map splits; - protected static Map fooFieldNames; - protected static Map fooPositionIndices; protected static Map> fooFunctions; static { @@ -246,30 +214,6 @@ public static class Boo { topicsToSchemas.put(PARTITIONED_TOPIC_5.getSchemaName(), Schema.JSON(TestPulsarMetadata.Foo.class).getSchemaInfo()); topicsToSchemas.put(PARTITIONED_TOPIC_6.getSchemaName(), Schema.JSON(TestPulsarMetadata.Foo.class).getSchemaInfo()); - fooTypes = new HashMap<>(); - fooTypes.put("field1", IntegerType.INTEGER); - fooTypes.put("field2", VarcharType.VARCHAR); - fooTypes.put("field3", RealType.REAL); - fooTypes.put("field4", DoubleType.DOUBLE); - fooTypes.put("field5", BooleanType.BOOLEAN); - fooTypes.put("field6", BigintType.BIGINT); - fooTypes.put("timestamp", TIMESTAMP); - fooTypes.put("time", TIME); - fooTypes.put("date", DATE); - fooTypes.put("bar.field1", IntegerType.INTEGER); - fooTypes.put("bar.field2", VarcharType.VARCHAR); - fooTypes.put("bar.test.field4", DoubleType.DOUBLE); - fooTypes.put("bar.test.field5", BooleanType.BOOLEAN); - fooTypes.put("bar.test.field6", BigintType.BIGINT); - fooTypes.put("bar.test.foobar.field1", IntegerType.INTEGER); - fooTypes.put("bar.field3", RealType.REAL); - fooTypes.put("bar.test2.field4", DoubleType.DOUBLE); - fooTypes.put("bar.test2.field5", BooleanType.BOOLEAN); - fooTypes.put("bar.test2.field6", BigintType.BIGINT); - fooTypes.put("bar.test2.foobar.field1", IntegerType.INTEGER); - // Enums currently map to VARCHAR - fooTypes.put("field7", VarcharType.VARCHAR); - topicsToNumEntries = new HashMap<>(); topicsToNumEntries.put(TOPIC_1.getSchemaName(), 1233L); topicsToNumEntries.put(TOPIC_2.getSchemaName(), 0L); @@ -286,267 +230,38 @@ public static class Boo { topicsToNumEntries.put(PARTITIONED_TOPIC_5.getSchemaName(), 800L); topicsToNumEntries.put(PARTITIONED_TOPIC_6.getSchemaName(), 1L); - fooFieldNames = new HashMap<>(); - fooPositionIndices = new HashMap<>(); - fooColumnHandles = new LinkedList<>(); - - String[] fieldNames1 = {"field1"}; - Integer[] positionIndices1 = {0}; - fooFieldNames.put("field1", fieldNames1); - fooPositionIndices.put("field1", positionIndices1); - fooColumnHandles.add(new PulsarColumnHandle(pulsarConnectorId.toString(), - "field1", - fooTypes.get("field1"), - false, - false, - fooFieldNames.get("field1"), - fooPositionIndices.get("field1"), null)); - - - String[] fieldNames2 = {"field2"}; - Integer[] positionIndices2 = {1}; - fooFieldNames.put("field2", fieldNames2); - fooPositionIndices.put("field2", positionIndices2); - fooColumnHandles.add(new PulsarColumnHandle(pulsarConnectorId.toString(), - "field2", - fooTypes.get("field2"), - false, - false, - fieldNames2, - positionIndices2, null)); - - String[] fieldNames3 = {"field3"}; - Integer[] positionIndices3 = {2}; - fooFieldNames.put("field3", fieldNames3); - fooPositionIndices.put("field3", positionIndices3); - fooColumnHandles.add(new PulsarColumnHandle(pulsarConnectorId.toString(), - "field3", - fooTypes.get("field3"), - false, - false, - fieldNames3, - positionIndices3, null)); - - String[] fieldNames4 = {"field4"}; - Integer[] positionIndices4 = {3}; - fooFieldNames.put("field4", fieldNames4); - fooPositionIndices.put("field4", positionIndices4); - fooColumnHandles.add(new PulsarColumnHandle(pulsarConnectorId.toString(), - "field4", - fooTypes.get("field4"), - false, - false, - fieldNames4, - positionIndices4, null)); - - - String[] fieldNames5 = {"field5"}; - Integer[] positionIndices5 = {4}; - fooFieldNames.put("field5", fieldNames5); - fooPositionIndices.put("field5", positionIndices5); - fooColumnHandles.add(new PulsarColumnHandle(pulsarConnectorId.toString(), - "field5", - fooTypes.get("field5"), - false, - false, - fieldNames5, - positionIndices5, null)); - - String[] fieldNames6 = {"field6"}; - Integer[] positionIndices6 = {5}; - fooFieldNames.put("field6", fieldNames6); - fooPositionIndices.put("field6", positionIndices6); - fooColumnHandles.add(new PulsarColumnHandle(pulsarConnectorId.toString(), - "field6", - fooTypes.get("field6"), - false, - false, - fieldNames6, - positionIndices6, null)); - - String[] fieldNames7 = {"timestamp"}; - Integer[] positionIndices7 = {6}; - fooFieldNames.put("timestamp", fieldNames7); - fooPositionIndices.put("timestamp", positionIndices7); - fooColumnHandles.add(new PulsarColumnHandle(pulsarConnectorId.toString(), - "timestamp", - fooTypes.get("timestamp"), - false, - false, - fieldNames7, - positionIndices7, null)); - - String[] fieldNames8 = {"time"}; - Integer[] positionIndices8 = {7}; - fooFieldNames.put("time", fieldNames8); - fooPositionIndices.put("time", positionIndices8); - fooColumnHandles.add(new PulsarColumnHandle(pulsarConnectorId.toString(), - "time", - fooTypes.get("time"), - false, - false, - fieldNames8, - positionIndices8, null)); - - String[] fieldNames9 = {"date"}; - Integer[] positionIndices9 = {8}; - fooFieldNames.put("date", fieldNames9); - fooPositionIndices.put("date", positionIndices9); - fooColumnHandles.add(new PulsarColumnHandle(pulsarConnectorId.toString(), - "date", - fooTypes.get("date"), - false, - false, - fieldNames9, - positionIndices9, null)); - - String[] bar_fieldNames1 = {"bar", "field1"}; - Integer[] bar_positionIndices1 = {9, 0}; - fooFieldNames.put("bar.field1", bar_fieldNames1); - fooPositionIndices.put("bar.field1", bar_positionIndices1); - fooColumnHandles.add(new PulsarColumnHandle(pulsarConnectorId.toString(), - "bar.field1", - fooTypes.get("bar.field1"), - false, - false, - bar_fieldNames1, - bar_positionIndices1, null)); - - String[] bar_fieldNames2 = {"bar", "field2"}; - Integer[] bar_positionIndices2 = {9, 1}; - fooFieldNames.put("bar.field2", bar_fieldNames2); - fooPositionIndices.put("bar.field2", bar_positionIndices2); - fooColumnHandles.add(new PulsarColumnHandle(pulsarConnectorId.toString(), - "bar.field2", - fooTypes.get("bar.field2"), - false, - false, - bar_fieldNames2, - bar_positionIndices2, null)); - - String[] bar_test_fieldNames4 = {"bar", "test", "field4"}; - Integer[] bar_test_positionIndices4 = {9, 2, 0}; - fooFieldNames.put("bar.test.field4", bar_test_fieldNames4); - fooPositionIndices.put("bar.test.field4", bar_test_positionIndices4); - fooColumnHandles.add(new PulsarColumnHandle(pulsarConnectorId.toString(), - "bar.test.field4", - fooTypes.get("bar.test.field4"), - false, - false, - bar_test_fieldNames4, - bar_test_positionIndices4, null)); - - String[] bar_test_fieldNames5 = {"bar", "test", "field5"}; - Integer[] bar_test_positionIndices5 = {9, 2, 1}; - fooFieldNames.put("bar.test.field5", bar_test_fieldNames5); - fooPositionIndices.put("bar.test.field5", bar_test_positionIndices5); - fooColumnHandles.add(new PulsarColumnHandle(pulsarConnectorId.toString(), - "bar.test.field5", - fooTypes.get("bar.test.field5"), - false, - false, - bar_test_fieldNames5, - bar_test_positionIndices5, null)); - - String[] bar_test_fieldNames6 = {"bar", "test", "field6"}; - Integer[] bar_test_positionIndices6 = {9, 2, 2}; - fooFieldNames.put("bar.test.field6", bar_test_fieldNames6); - fooPositionIndices.put("bar.test.field6", bar_test_positionIndices6); - fooColumnHandles.add(new PulsarColumnHandle(pulsarConnectorId.toString(), - "bar.test.field6", - fooTypes.get("bar.test.field6"), - false, - false, - bar_test_fieldNames6, - bar_test_positionIndices6, null)); - - String[] bar_test_foobar_fieldNames1 = {"bar", "test", "foobar", "field1"}; - Integer[] bar_test_foobar_positionIndices1 = {9, 2, 6, 0}; - fooFieldNames.put("bar.test.foobar.field1", bar_test_foobar_fieldNames1); - fooPositionIndices.put("bar.test.foobar.field1", bar_test_foobar_positionIndices1); - fooColumnHandles.add(new PulsarColumnHandle(pulsarConnectorId.toString(), - "bar.test.foobar.field1", - fooTypes.get("bar.test.foobar.field1"), - false, - false, - bar_test_foobar_fieldNames1, - bar_test_foobar_positionIndices1, null)); - - String[] bar_field3 = {"bar", "field3"}; - Integer[] bar_positionIndices3 = {9, 3}; - fooFieldNames.put("bar.field3", bar_field3); - fooPositionIndices.put("bar.field3", bar_positionIndices3); - fooColumnHandles.add(new PulsarColumnHandle(pulsarConnectorId.toString(), - "bar.field3", - fooTypes.get("bar.field3"), - false, - false, - bar_field3, - bar_positionIndices3, null)); - - String[] bar_test2_fieldNames4 = {"bar", "test2", "field4"}; - Integer[] bar_test2_positionIndices4 = {9, 4, 0}; - fooFieldNames.put("bar.test2.field4", bar_test2_fieldNames4); - fooPositionIndices.put("bar.test2.field4", bar_test2_positionIndices4); - fooColumnHandles.add(new PulsarColumnHandle(pulsarConnectorId.toString(), - "bar.test2.field4", - fooTypes.get("bar.test2.field4"), - false, - false, - bar_test2_fieldNames4, - bar_test2_positionIndices4, null)); - - String[] bar_test2_fieldNames5 = {"bar", "test2", "field5"}; - Integer[] bar_test2_positionIndices5 = {9, 4, 1}; - fooFieldNames.put("bar.test2.field5", bar_test2_fieldNames5); - fooPositionIndices.put("bar.test2.field5", bar_test2_positionIndices5); - fooColumnHandles.add(new PulsarColumnHandle(pulsarConnectorId.toString(), - "bar.test2.field5", - fooTypes.get("bar.test2.field5"), - false, - false, - bar_test2_fieldNames5, - bar_test2_positionIndices5, null)); - - String[] bar_test2_fieldNames6 = {"bar", "test2", "field6"}; - Integer[] bar_test2_positionIndices6 = {9, 4, 2}; - fooFieldNames.put("bar.test2.field6", bar_test2_fieldNames6); - fooPositionIndices.put("bar.test2.field6", bar_test2_positionIndices6); - fooColumnHandles.add(new PulsarColumnHandle(pulsarConnectorId.toString(), - "bar.test2.field6", - fooTypes.get("bar.test2.field6"), - false, - false, - bar_test2_fieldNames6, - bar_test2_positionIndices6, null)); - - String[] bar_test2_foobar_fieldNames1 = {"bar", "test2", "foobar", "field1"}; - Integer[] bar_test2_foobar_positionIndices1 = {9, 4, 6, 0}; - fooFieldNames.put("bar.test2.foobar.field1", bar_test2_foobar_fieldNames1); - fooPositionIndices.put("bar.test2.foobar.field1", bar_test2_foobar_positionIndices1); - fooColumnHandles.add(new PulsarColumnHandle(pulsarConnectorId.toString(), - "bar.test2.foobar.field1", - fooTypes.get("bar.test2.foobar.field1"), - false, - false, - bar_test2_foobar_fieldNames1, - bar_test2_foobar_positionIndices1, null)); - - String[] fieldNames10 = {"field7"}; - Integer[] positionIndices10 = {10}; - fooFieldNames.put("field7", fieldNames10); - fooPositionIndices.put("field7", positionIndices10); - fooColumnHandles.add(new PulsarColumnHandle(pulsarConnectorId.toString(), - "field7", - fooTypes.get("field7"), - false, - false, - fieldNames10, - positionIndices10, null)); - - fooColumnHandles.addAll(PulsarInternalColumn.getInternalFields().stream() - .map(pulsarInternalColumn -> pulsarInternalColumn.getColumnHandle(pulsarConnectorId.toString(), false)) - .collect(Collectors.toList())); + + fooFieldNames.add("field1"); + fooFieldNames.add("field2"); + fooFieldNames.add("field3"); + fooFieldNames.add("field4"); + fooFieldNames.add("field5"); + fooFieldNames.add("field6"); + fooFieldNames.add("timestamp"); + fooFieldNames.add("time"); + fooFieldNames.add("date"); + fooFieldNames.add("bar"); + fooFieldNames.add("field7"); + + + ConnectorContext prestoConnectorContext = new TestingConnectorContext(); + dispatchingRowDecoderFactory = new PulsarDispatchingRowDecoderFactory(prestoConnectorContext.getTypeManager()); + + topicsToColumnHandles.put(PARTITIONED_TOPIC_1, getColumnColumnHandles(PARTITIONED_TOPIC_1,topicsToSchemas.get(PARTITIONED_TOPIC_1.getSchemaName()), PulsarColumnHandle.HandleKeyValueType.NONE,true)); + topicsToColumnHandles.put(PARTITIONED_TOPIC_2, getColumnColumnHandles(PARTITIONED_TOPIC_2,topicsToSchemas.get(PARTITIONED_TOPIC_2.getSchemaName()), PulsarColumnHandle.HandleKeyValueType.NONE,true)); + topicsToColumnHandles.put(PARTITIONED_TOPIC_3, getColumnColumnHandles(PARTITIONED_TOPIC_3,topicsToSchemas.get(PARTITIONED_TOPIC_3.getSchemaName()), PulsarColumnHandle.HandleKeyValueType.NONE,true)); + topicsToColumnHandles.put(PARTITIONED_TOPIC_4, getColumnColumnHandles(PARTITIONED_TOPIC_4,topicsToSchemas.get(PARTITIONED_TOPIC_4.getSchemaName()), PulsarColumnHandle.HandleKeyValueType.NONE,true)); + topicsToColumnHandles.put(PARTITIONED_TOPIC_5, getColumnColumnHandles(PARTITIONED_TOPIC_5,topicsToSchemas.get(PARTITIONED_TOPIC_5.getSchemaName()), PulsarColumnHandle.HandleKeyValueType.NONE,true)); + topicsToColumnHandles.put(PARTITIONED_TOPIC_6, getColumnColumnHandles(PARTITIONED_TOPIC_6,topicsToSchemas.get(PARTITIONED_TOPIC_6.getSchemaName()), PulsarColumnHandle.HandleKeyValueType.NONE,true)); + + + topicsToColumnHandles.put(TOPIC_1, getColumnColumnHandles(TOPIC_1,topicsToSchemas.get(TOPIC_1.getSchemaName()), PulsarColumnHandle.HandleKeyValueType.NONE,true)); + topicsToColumnHandles.put(TOPIC_2, getColumnColumnHandles(TOPIC_2,topicsToSchemas.get(TOPIC_2.getSchemaName()), PulsarColumnHandle.HandleKeyValueType.NONE,true)); + topicsToColumnHandles.put(TOPIC_3, getColumnColumnHandles(TOPIC_3,topicsToSchemas.get(TOPIC_3.getSchemaName()), PulsarColumnHandle.HandleKeyValueType.NONE,true)); + topicsToColumnHandles.put(TOPIC_4, getColumnColumnHandles(TOPIC_4,topicsToSchemas.get(TOPIC_4.getSchemaName()), PulsarColumnHandle.HandleKeyValueType.NONE,true)); + topicsToColumnHandles.put(TOPIC_5, getColumnColumnHandles(TOPIC_5,topicsToSchemas.get(TOPIC_5.getSchemaName()), PulsarColumnHandle.HandleKeyValueType.NONE,true)); + topicsToColumnHandles.put(TOPIC_6, getColumnColumnHandles(TOPIC_6,topicsToSchemas.get(TOPIC_6.getSchemaName()), PulsarColumnHandle.HandleKeyValueType.NONE,true)); + splits = new HashMap<>(); @@ -554,6 +269,7 @@ public static class Boo { allTopics.addAll(topicNames); allTopics.addAll(partitionedTopicNames); + for (TopicName topicName : allTopics) { if (topicsToSchemas.containsKey(topicName.getSchemaName())) { splits.put(topicName, new PulsarSplit(0, pulsarConnectorId.toString(), @@ -590,15 +306,6 @@ public static class Boo { fooFunctions.put("bar.field2", integer -> integer % 2 == 0 ? null : String.valueOf(integer + 2)); fooFunctions.put("bar.field3", integer -> integer + 3.0f); - fooFunctions.put("bar.test.field4", integer -> integer + 1.0); - fooFunctions.put("bar.test.field5", integer -> (integer + 1) % 2 == 0); - fooFunctions.put("bar.test.field6", integer -> integer + 10L); - fooFunctions.put("bar.test.foobar.field1", integer -> integer % 3); - - fooFunctions.put("bar.test2.field4", integer -> integer + 2.0); - fooFunctions.put("bar.test2.field5", integer -> (integer + 1) % 32 == 0); - fooFunctions.put("bar.test2.field6", integer -> integer + 15L); - fooFunctions.put("bar.test2.foobar.field1", integer -> integer % 3); fooFunctions.put("field7", integer -> Foo.TestEnum.values()[integer % Foo.TestEnum.values().length]); } catch (Throwable e) { @@ -607,6 +314,53 @@ public static class Boo { } } + + /** + * Parse PulsarColumnMetadata to PulsarColumnHandle Util + * @param schemaInfo + * @param handleKeyValueType + * @param includeInternalColumn + * @param dispatchingRowDecoderFactory + * @return + */ + protected static List getColumnColumnHandles(TopicName topicName, SchemaInfo schemaInfo, + PulsarColumnHandle.HandleKeyValueType handleKeyValueType, boolean includeInternalColumn) { + List columnHandles = new ArrayList<>(); + List columnMetadata = mockColumnMetadata().getPulsarColumns(topicName, schemaInfo, + includeInternalColumn, handleKeyValueType); + columnMetadata.forEach(column -> { + PulsarColumnMetadata pulsarColumnMetadata = (PulsarColumnMetadata) column; + columnHandles.add(new PulsarColumnHandle( + pulsarConnectorId.toString(), + pulsarColumnMetadata.getNameWithCase(), + pulsarColumnMetadata.getType(), + pulsarColumnMetadata.isHidden(), + pulsarColumnMetadata.isInternal(), + pulsarColumnMetadata.getDecoderExtraInfo().getMapping(), + pulsarColumnMetadata.getDecoderExtraInfo().getDataFormat(), pulsarColumnMetadata.getDecoderExtraInfo().getFormatHint(), + pulsarColumnMetadata.getHandleKeyValueType())); + + }); + return columnHandles; + } + + public static PulsarMetadata mockColumnMetadata() { + ConnectorContext prestoConnectorContext = new TestingConnectorContext(); + PulsarConnectorConfig pulsarConnectorConfig = spy(new PulsarConnectorConfig()); + pulsarConnectorConfig.setMaxEntryReadBatchSize(1); + pulsarConnectorConfig.setMaxSplitEntryQueueSize(10); + pulsarConnectorConfig.setMaxSplitMessageQueueSize(100); + PulsarDispatchingRowDecoderFactory dispatchingRowDecoderFactory = + new PulsarDispatchingRowDecoderFactory(prestoConnectorContext.getTypeManager()); + PulsarMetadata pulsarMetadata = new PulsarMetadata(pulsarConnectorId, pulsarConnectorConfig, dispatchingRowDecoderFactory); + return pulsarMetadata; + } + + public static PulsarConnectorId getPulsarConnectorId() { + assertNotNull(pulsarConnectorId); + return pulsarConnectorId; + } + private static List getTopicEntries(String topicSchemaName) { List entries = new LinkedList<>(); @@ -765,7 +519,7 @@ public SchemaInfo answer(InvocationOnMock invocationOnMock) throws Throwable { doReturn(schemas).when(pulsarAdmin).schemas(); doReturn(pulsarAdmin).when(this.pulsarConnectorConfig).getPulsarAdmin(); - this.pulsarMetadata = new PulsarMetadata(pulsarConnectorId, this.pulsarConnectorConfig); + this.pulsarMetadata = new PulsarMetadata(pulsarConnectorId, this.pulsarConnectorConfig, dispatchingRowDecoderFactory); this.pulsarSplitManager = Mockito.spy(new PulsarSplitManager(pulsarConnectorId, this.pulsarConnectorConfig)); ManagedLedgerFactory managedLedgerFactory = mock(ManagedLedgerFactory.class); @@ -821,36 +575,14 @@ public Object answer(InvocationOnMock invocationOnMock) throws Throwable { new Thread(new Runnable() { @Override public void run() { - List < Entry > entries = new LinkedList<>(); + List entries = new LinkedList<>(); for (int i = 0; i < readEntries; i++) { - Foo.Bar foobar = new Foo.Bar(); - foobar.field1 = (int) fooFunctions.get("bar.test.foobar.field1").apply(count); - - Boo boo1 = new Boo(); - boo1.field4 = (double) fooFunctions.get("bar.test.field4").apply(count); - boo1.field5 = (boolean) fooFunctions.get("bar.test.field5").apply(count); - boo1.field6 = (long) fooFunctions.get("bar.test.field6").apply(count); - boo1.foo = new Foo(); - boo1.boo = null; - boo1.bar = new Bar(); - boo1.foobar = foobar; - - Boo boo2 = new Boo(); - boo2.field4 = (double) fooFunctions.get("bar.test2.field4").apply(count); - boo2.field5 = (boolean) fooFunctions.get("bar.test2.field5").apply(count); - boo2.field6 = (long) fooFunctions.get("bar.test2.field6").apply(count); - boo2.foo = new Foo(); - boo2.boo = boo1; - boo2.bar = new Bar(); - boo2.foobar = foobar; - TestPulsarConnector.Bar bar = new TestPulsarConnector.Bar(); bar.field1 = fooFunctions.get("bar.field1").apply(count) == null ? null : (int) fooFunctions.get("bar.field1").apply(count); bar.field2 = fooFunctions.get("bar.field2").apply(count) == null ? null : (String) fooFunctions.get("bar.field2").apply(count); bar.field3 = (float) fooFunctions.get("bar.field3").apply(count); - bar.test = boo1; - bar.test2 = count % 2 == 0 ? null : boo2; + Foo foo = new Foo(); foo.field1 = (int) fooFunctions.get("field1").apply(count); @@ -946,11 +678,10 @@ public Long answer(InvocationOnMock invocationOnMock) throws Throwable { when(PulsarConnectorCache.instance.getManagedLedgerFactory()).thenReturn(managedLedgerFactory); for (Map.Entry split : splits.entrySet()) { - PulsarRecordCursor pulsarRecordCursor = spy(new PulsarRecordCursor( - fooColumnHandles, split.getValue(), + topicsToColumnHandles.get(split.getKey()), split.getValue(), pulsarConnectorConfig, managedLedgerFactory, new ManagedLedgerConfig(), - new PulsarConnectorMetricsTracker(new NullStatsProvider()))); + new PulsarConnectorMetricsTracker(new NullStatsProvider()),dispatchingRowDecoderFactory)); this.pulsarRecordCursors.put(split.getKey(), pulsarRecordCursor); } } diff --git a/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/TestPulsarConnectorConfig.java b/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/TestPulsarConnectorConfig.java index 0b2609a..2471890 100644 --- a/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/TestPulsarConnectorConfig.java +++ b/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/TestPulsarConnectorConfig.java @@ -94,7 +94,7 @@ public void testGetOffloadPolices() throws Exception { Assert.assertNotNull(offloadPolicies); Assert.assertEquals(offloadPolicies.getManagedLedgerOffloadDriver(), managedLedgerOffloadDriver); Assert.assertEquals(offloadPolicies.getOffloadersDirectory(), offloaderDirectory); - Assert.assertEquals(offloadPolicies.getManagedLedgerOffloadMaxThreads(), managedLedgerOffloadMaxThreads); + Assert.assertEquals((int) offloadPolicies.getManagedLedgerOffloadMaxThreads(), (int) managedLedgerOffloadMaxThreads); Assert.assertEquals(offloadPolicies.getS3ManagedLedgerOffloadBucket(), bucket); Assert.assertEquals(offloadPolicies.getS3ManagedLedgerOffloadRegion(), region); Assert.assertEquals(offloadPolicies.getS3ManagedLedgerOffloadServiceEndpoint(), endpoint); diff --git a/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/TestPulsarKeyValueSchemaHandler.java b/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/TestPulsarKeyValueSchemaHandler.java deleted file mode 100644 index 1fe881e..0000000 --- a/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/TestPulsarKeyValueSchemaHandler.java +++ /dev/null @@ -1,353 +0,0 @@ -/** - * 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 com.fasterxml.jackson.databind.ObjectMapper; -import io.netty.buffer.ByteBuf; -import io.netty.buffer.Unpooled; -import io.prestosql.spi.connector.ColumnMetadata; -import java.io.IOException; -import java.util.LinkedList; -import java.util.List; -import java.util.Objects; -import java.util.Optional; -import lombok.Data; -import lombok.extern.slf4j.Slf4j; -import org.apache.pulsar.client.api.Schema; -import org.apache.pulsar.client.impl.schema.KeyValueSchema; -import org.apache.pulsar.client.impl.schema.KeyValueSchemaInfo; -import org.apache.pulsar.common.api.raw.RawMessage; -import org.apache.pulsar.common.api.raw.RawMessageImpl; -import org.apache.pulsar.common.naming.TopicName; -import org.apache.pulsar.common.schema.KeyValue; -import org.apache.pulsar.common.schema.KeyValueEncodingType; -import org.apache.pulsar.common.schema.SchemaInfo; -import org.mockito.Mockito; -import org.testng.Assert; -import org.testng.annotations.Test; - -import static org.mockito.Mockito.mock; - - -/** - * Unit test for KeyValueSchemaHandler - */ -@Slf4j -public class TestPulsarKeyValueSchemaHandler { - - private final static ObjectMapper objectMapper = new ObjectMapper(); - - private Schema> schema1 = - Schema.KeyValue(Schema.STRING, Schema.INT32, KeyValueEncodingType.INLINE); - - private Schema> schema2 = - Schema.KeyValue(Schema.STRING, Schema.JSON(Foo.class), KeyValueEncodingType.INLINE); - - private Schema> schema3 = - Schema.KeyValue(Schema.AVRO(Boo.class), Schema.INT64, KeyValueEncodingType.SEPARATED); - - private Schema> schema4 = - Schema.KeyValue(Schema.JSON(Boo.class), Schema.AVRO(Foo.class), KeyValueEncodingType.SEPARATED); - - private final static TopicName topicName = TopicName.get("persistent://public/default/kv-test"); - - private final static Foo foo; - - private final static Boo boo; - - private final Integer KEY_FIELD_NAME_PREFIX_LENGTH = PulsarColumnMetadata.KEY_SCHEMA_COLUMN_PREFIX.length(); - - static { - foo = new Foo(); - foo.field1 = "field1-value"; - foo.field2 = 20; - - boo = new Boo(); - boo.field1 = "field1-value"; - boo.field2 = true; - boo.field3 = 10.2; - } - - - @Test - public void testSchema1() throws IOException { - final String keyData = "test-key"; - final Integer valueData = 10; - List columnMetadataList = - PulsarMetadata.getPulsarColumns(topicName, schema1.getSchemaInfo(), - true, null); - int keyCount = 0; - int valueCount = 0; - for (ColumnMetadata columnMetadata : columnMetadataList) { - PulsarColumnMetadata pulsarColumnMetadata = (PulsarColumnMetadata) columnMetadata; - if (pulsarColumnMetadata.isKey()) { - keyCount++; - } else if (pulsarColumnMetadata.isValue()) { - valueCount++; - } - } - Assert.assertEquals(keyCount, 1); - Assert.assertEquals(valueCount, 1); - - List columnHandleList = getColumnHandlerList(columnMetadataList); - - KeyValueSchemaHandler keyValueSchemaHandler = - new KeyValueSchemaHandler(null, null,schema1.getSchemaInfo(), columnHandleList); - - RawMessageImpl message = mock(RawMessageImpl.class); - Mockito.when(message.getData()).thenReturn( - Unpooled.wrappedBuffer(schema1.encode(new KeyValue<>(keyData, valueData))) - ); - - KeyValue byteBufKeyValue = getKeyValueByteBuf(message, schema1); - Object object = keyValueSchemaHandler.deserialize(byteBufKeyValue.getKey(), byteBufKeyValue.getValue(), null); - Assert.assertEquals(keyValueSchemaHandler.extractField(0, object), keyData); - Assert.assertEquals(keyValueSchemaHandler.extractField(1, object), valueData); - } - - @Test - public void testSchema2() throws IOException { - final String keyData = "test-key"; - - List columnMetadataList = - PulsarMetadata.getPulsarColumns(topicName, schema2.getSchemaInfo(), - true, null); - int keyCount = 0; - int valueCount = 0; - for (ColumnMetadata columnMetadata : columnMetadataList) { - PulsarColumnMetadata pulsarColumnMetadata = (PulsarColumnMetadata) columnMetadata; - if (pulsarColumnMetadata.isKey()) { - keyCount++; - } else if (pulsarColumnMetadata.isValue()) { - valueCount++; - } - } - Assert.assertEquals(keyCount, 1); - Assert.assertEquals(valueCount, 2); - - List columnHandleList = getColumnHandlerList(columnMetadataList); - - RawMessage message = mock(RawMessage.class); - Mockito.when(message.getData()).thenReturn( - Unpooled.wrappedBuffer(schema2.encode(new KeyValue<>(keyData, foo))) - ); - - - KeyValueSchemaHandler keyValueSchemaHandler = - new KeyValueSchemaHandler(null, null, schema2.getSchemaInfo(), columnHandleList); - - KeyValue byteBufKeyValue = getKeyValueByteBuf(message, schema2); - Object object = keyValueSchemaHandler.deserialize(byteBufKeyValue.getKey(), byteBufKeyValue.getValue(), null); - Assert.assertEquals(keyValueSchemaHandler.extractField(0, object), keyData); - Assert.assertEquals(keyValueSchemaHandler.extractField(1, object), - foo.getValue(columnHandleList.get(1).getName())); - Assert.assertEquals(keyValueSchemaHandler.extractField(2, object), - foo.getValue(columnHandleList.get(2).getName())); - } - - @Test - public void testSchema3() throws IOException { - final Boo boo = new Boo(); - boo.field1 = "field1-value"; - boo.field2 = true; - boo.field3 = 10.2; - final Long valueData = 999999L; - - List columnMetadataList = - PulsarMetadata.getPulsarColumns(topicName, schema3.getSchemaInfo(), - true, null); - int keyCount = 0; - int valueCount = 0; - for (ColumnMetadata columnMetadata : columnMetadataList) { - PulsarColumnMetadata pulsarColumnMetadata = (PulsarColumnMetadata) columnMetadata; - if (pulsarColumnMetadata.isKey()) { - keyCount++; - } else if (pulsarColumnMetadata.isValue()) { - valueCount++; - } - } - Assert.assertEquals(keyCount, 3); - Assert.assertEquals(valueCount, 1); - - List columnHandleList = getColumnHandlerList(columnMetadataList); - - PulsarSqlSchemaInfoProvider pulsarSqlSchemaInfoProvider = mock(PulsarSqlSchemaInfoProvider.class); - - KeyValue kvSchemaInfo = - KeyValueSchemaInfo.decodeKeyValueSchemaInfo(schema3.getSchemaInfo()); - - AvroSchemaHandler avroSchemaHandler = - new AvroSchemaHandler(pulsarSqlSchemaInfoProvider, kvSchemaInfo.getKey(), columnHandleList); - PulsarPrimitiveSchemaHandler pulsarPrimitiveSchemaHandler = - new PulsarPrimitiveSchemaHandler(kvSchemaInfo.getValue()); - KeyValueSchemaHandler keyValueSchemaHandler = - new KeyValueSchemaHandler(avroSchemaHandler, pulsarPrimitiveSchemaHandler, columnHandleList); - - RawMessage message = mock(RawMessage.class); - Mockito.when(message.getKeyBytes()).thenReturn( - Optional.of(Unpooled.wrappedBuffer( - ((KeyValueSchema) schema3).getKeySchema().encode(boo) - )) - ); - Mockito.when(message.getData()).thenReturn( - Unpooled.wrappedBuffer(schema3.encode(new KeyValue<>(boo, valueData))) - ); - - KeyValue byteBufKeyValue = getKeyValueByteBuf(message, schema3); - Integer a = 1; - Object object = keyValueSchemaHandler.deserialize(byteBufKeyValue.getKey(), byteBufKeyValue.getValue(), null); - Assert.assertEquals(keyValueSchemaHandler.extractField(0, object).toString(), - boo.getValue(columnHandleList.get(0).getName().substring(KEY_FIELD_NAME_PREFIX_LENGTH))); - Assert.assertEquals(keyValueSchemaHandler.extractField(1, object), - boo.getValue(columnHandleList.get(1).getName().substring(KEY_FIELD_NAME_PREFIX_LENGTH))); - Assert.assertEquals(keyValueSchemaHandler.extractField(2, object), - boo.getValue(columnHandleList.get(2).getName().substring(KEY_FIELD_NAME_PREFIX_LENGTH))); - Assert.assertEquals(keyValueSchemaHandler.extractField(3, object), valueData); - } - - @Test - public void testSchema4() throws IOException { - List columnMetadataList = - PulsarMetadata.getPulsarColumns(topicName, schema4.getSchemaInfo(), - true, null); - int keyCount = 0; - int valueCount = 0; - for (ColumnMetadata columnMetadata : columnMetadataList) { - PulsarColumnMetadata pulsarColumnMetadata = (PulsarColumnMetadata) columnMetadata; - if (pulsarColumnMetadata.isKey()) { - keyCount++; - } else if (pulsarColumnMetadata.isValue()) { - valueCount++; - } - } - Assert.assertEquals(keyCount, 3); - Assert.assertEquals(valueCount, 2); - - List columnHandleList = getColumnHandlerList(columnMetadataList); - - PulsarSqlSchemaInfoProvider pulsarSqlSchemaInfoProvider = mock(PulsarSqlSchemaInfoProvider.class); - - KeyValue kvSchemaInfo = - KeyValueSchemaInfo.decodeKeyValueSchemaInfo(schema4.getSchemaInfo()); - - AvroSchemaHandler avroSchemaHandler = - new AvroSchemaHandler(pulsarSqlSchemaInfoProvider, kvSchemaInfo.getValue(), columnHandleList); - JSONSchemaHandler jsonSchemaHandler = new JSONSchemaHandler(columnHandleList); - KeyValueSchemaHandler keyValueSchemaHandler = - new KeyValueSchemaHandler(jsonSchemaHandler, avroSchemaHandler, columnHandleList); - - - RawMessage message = mock(RawMessage.class); - Mockito.when(message.getKeyBytes()).thenReturn( - Optional.of(Unpooled.wrappedBuffer( - ((KeyValueSchema) schema4).getKeySchema().encode(boo) - )) - ); - Mockito.when(message.getData()).thenReturn( - Unpooled.wrappedBuffer(schema4.encode(new KeyValue<>(boo, foo))) - ); - - KeyValue byteBufKeyValue = getKeyValueByteBuf(message, schema4); - Object object = keyValueSchemaHandler.deserialize(byteBufKeyValue.getKey(), byteBufKeyValue.getValue(), null); - Assert.assertEquals(keyValueSchemaHandler.extractField(0, object).toString(), - boo.getValue(columnHandleList.get(0).getName().substring(KEY_FIELD_NAME_PREFIX_LENGTH))); - Assert.assertEquals(keyValueSchemaHandler.extractField(1, object), - boo.getValue(columnHandleList.get(1).getName().substring(KEY_FIELD_NAME_PREFIX_LENGTH))); - Assert.assertEquals(keyValueSchemaHandler.extractField(2, object), - boo.getValue(columnHandleList.get(2).getName().substring(KEY_FIELD_NAME_PREFIX_LENGTH))); - Assert.assertEquals(keyValueSchemaHandler.extractField(3, object).toString(), - foo.getValue(columnHandleList.get(3).getName())); - Assert.assertEquals(keyValueSchemaHandler.extractField(4, object).toString(), - foo.getValue(columnHandleList.get(4).getName()) + ""); - } - - private List getColumnHandlerList(List columnMetadataList) { - List columnHandleList = new LinkedList<>(); - - columnMetadataList.forEach(columnMetadata -> { - PulsarColumnMetadata pulsarColumnMetadata = (PulsarColumnMetadata) columnMetadata; - PulsarColumnHandle pulsarColumnHandle = new PulsarColumnHandle( - "connectorId", - pulsarColumnMetadata.getNameWithCase(), - pulsarColumnMetadata.getType(), - pulsarColumnMetadata.isHidden(), - pulsarColumnMetadata.isInternal(), - pulsarColumnMetadata.getFieldNames(), - pulsarColumnMetadata.getPositionIndices(), - pulsarColumnMetadata.getHandleKeyValueType()); - columnHandleList.add(pulsarColumnHandle); - }); - - return columnHandleList; - } - - public KeyValue getKeyValueByteBuf(RawMessage message, Schema schema) { - KeyValueEncodingType encodingType = KeyValueSchemaInfo.decodeKeyValueEncodingType(schema.getSchemaInfo()); - ByteBuf keyByteBuf = null; - if (Objects.equals(KeyValueEncodingType.SEPARATED, encodingType)) { - if (message.getKeyBytes().isPresent()) { - keyByteBuf = message.getKeyBytes().get(); - } else { - keyByteBuf = null; - } - } else { - keyByteBuf = null; - } - return new KeyValue<>(keyByteBuf, Unpooled.wrappedBuffer(message.getData())); - } - - @Data - static class Foo { - private String field1; - private Integer field2; - - public Object getValue(String fieldName) { - switch (fieldName) { - case "field1": - return field1; - case "field2": - return field2 == null ? null : new Long(field2); - default: - return null; - } - } - } - - @Data - static class Boo { - private String field1; - private Boolean field2; - private Double field3; - - public Object getValue(String fieldName) { - switch (fieldName) { - case "field1": - return field1; - case "field2": - return field2; - case "field3": - return field3 == null ? null : field3.doubleValue(); - default: - return null; - } - } - - } - -} diff --git a/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/TestPulsarMetadata.java b/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/TestPulsarMetadata.java index f137c8a..6e4b0de 100644 --- a/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/TestPulsarMetadata.java +++ b/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/TestPulsarMetadata.java @@ -20,44 +20,23 @@ import io.airlift.log.Logger; import io.prestosql.spi.PrestoException; -import io.prestosql.spi.connector.ColumnHandle; -import io.prestosql.spi.connector.ColumnMetadata; -import io.prestosql.spi.connector.ConnectorSession; -import io.prestosql.spi.connector.ConnectorTableHandle; -import io.prestosql.spi.connector.ConnectorTableMetadata; -import io.prestosql.spi.connector.SchemaTableName; -import io.prestosql.spi.connector.SchemaTablePrefix; -import io.prestosql.spi.connector.TableNotFoundException; -import java.util.Optional; -import org.apache.avro.Schema; +import io.prestosql.spi.connector.*; import org.apache.commons.lang3.StringUtils; import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.schema.SchemaInfo; -import javax.ws.rs.ClientErrorException; -import javax.ws.rs.core.Response; - import org.apache.pulsar.common.schema.SchemaType; import org.testng.annotations.Test; -import java.util.Arrays; -import java.util.HashMap; -import java.util.HashSet; -import java.util.LinkedList; -import java.util.List; -import java.util.Map; +import javax.ws.rs.ClientErrorException; +import javax.ws.rs.core.Response; +import java.util.*; import java.util.stream.Collectors; import static io.prestosql.spi.StandardErrorCode.NOT_FOUND; import static io.prestosql.spi.StandardErrorCode.NOT_SUPPORTED; -import static org.mockito.Mockito.eq; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; -import static org.testng.Assert.assertEquals; -import static org.testng.Assert.assertFalse; -import static org.testng.Assert.assertNotNull; -import static org.testng.Assert.assertTrue; -import static org.testng.Assert.fail; +import static org.mockito.Mockito.*; +import static org.testng.Assert.*; public class TestPulsarMetadata extends TestPulsarConnector { @@ -116,6 +95,8 @@ public void testGetTableMetadata(String delimiter) { topic.getLocalName() ); + List fooColumnHandles = topicsToColumnHandles.get(topic); + ConnectorTableMetadata tableMetadata = this.pulsarMetadata.getTableMetadata(mock(ConnectorSession.class), pulsarTableHandle); @@ -124,7 +105,7 @@ public void testGetTableMetadata(String delimiter) { assertEquals(tableMetadata.getColumns().size(), fooColumnHandles.size()); - List fieldNames = new LinkedList<>(fooFieldNames.keySet()); + List fieldNames = new LinkedList<>(fooFieldNames); for (PulsarInternalColumn internalField : PulsarInternalColumn.getInternalFields()) { fieldNames.add(internalField.getName()); @@ -286,7 +267,7 @@ public void testGetColumnHandles(String delimiter) { Map columnHandleMap = new HashMap<>(this.pulsarMetadata.getColumnHandles(mock(ConnectorSession.class), pulsarTableHandle)); - List fieldNames = new LinkedList<>(fooFieldNames.keySet()); + List fieldNames = new LinkedList<>(fooFieldNames); for (PulsarInternalColumn internalField : PulsarInternalColumn.getInternalFields()) { fieldNames.add(internalField.getName()); @@ -300,13 +281,8 @@ public void testGetColumnHandles(String delimiter) { assertEquals(pulsarColumnHandle, pulsarInternalColumn.getColumnHandle(pulsarConnectorId.toString(), false)); } else { - Schema schema = new Schema.Parser().parse(new String(topicsToSchemas.get(TOPIC_1.getSchemaName()) - .getSchema())); assertEquals(pulsarColumnHandle.getConnectorId(), pulsarConnectorId.toString()); assertEquals(pulsarColumnHandle.getName(), field); - assertEquals(pulsarColumnHandle.getPositionIndices(), fooPositionIndices.get(field)); - assertEquals(pulsarColumnHandle.getFieldNames(), fooFieldNames.get(field)); - assertEquals(pulsarColumnHandle.getType(), fooTypes.get(field)); assertFalse(pulsarColumnHandle.isHidden()); } columnHandleMap.remove(field); @@ -326,9 +302,9 @@ public void testListTableColumns(String delimiter) { = tableColumnsMap.get(new SchemaTableName(TOPIC_1.getNamespace(), TOPIC_1.getLocalName())); assertNotNull(columnMetadataList); assertEquals(columnMetadataList.size(), - fooColumnHandles.size()); + topicsToColumnHandles.get(TOPIC_1).size()); - List fieldNames = new LinkedList<>(fooFieldNames.keySet()); + List fieldNames = new LinkedList<>(fooFieldNames); for (PulsarInternalColumn internalField : PulsarInternalColumn.getInternalFields()) { fieldNames.add(internalField.getName()); @@ -349,9 +325,9 @@ public void testListTableColumns(String delimiter) { columnMetadataList = tableColumnsMap.get(new SchemaTableName(TOPIC_2.getNamespace(), TOPIC_2.getLocalName())); assertNotNull(columnMetadataList); assertEquals(columnMetadataList.size(), - fooColumnHandles.size()); + topicsToColumnHandles.get(TOPIC_2).size()); - fieldNames = new LinkedList<>(fooFieldNames.keySet()); + fieldNames = new LinkedList<>(fooFieldNames); for (PulsarInternalColumn internalField : PulsarInternalColumn.getInternalFields()) { fieldNames.add(internalField.getName()); @@ -378,9 +354,9 @@ public void testListTableColumns(String delimiter) { columnMetadataList = tableColumnsMap.get(new SchemaTableName(TOPIC_4.getNamespace(), TOPIC_4.getLocalName())); assertNotNull(columnMetadataList); assertEquals(columnMetadataList.size(), - fooColumnHandles.size()); + topicsToColumnHandles.get(TOPIC_4).size()); - fieldNames = new LinkedList<>(fooFieldNames.keySet()); + fieldNames = new LinkedList<>(fooFieldNames); for (PulsarInternalColumn internalField : PulsarInternalColumn.getInternalFields()) { fieldNames.add(internalField.getName()); diff --git a/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/TestPulsarPrimitiveSchemaHandler.java b/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/TestPulsarPrimitiveSchemaHandler.java deleted file mode 100644 index fc33077..0000000 --- a/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/TestPulsarPrimitiveSchemaHandler.java +++ /dev/null @@ -1,164 +0,0 @@ -/** - * 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.netty.buffer.ByteBufAllocator; -import io.prestosql.spi.connector.ColumnMetadata; -import lombok.extern.slf4j.Slf4j; - -import org.apache.pulsar.client.impl.schema.BooleanSchema; -import org.apache.pulsar.client.impl.schema.ByteSchema; -import org.apache.pulsar.client.impl.schema.BytesSchema; -import org.apache.pulsar.client.impl.schema.DateSchema; -import org.apache.pulsar.client.impl.schema.DoubleSchema; -import org.apache.pulsar.client.impl.schema.FloatSchema; -import org.apache.pulsar.client.impl.schema.IntSchema; -import org.apache.pulsar.client.impl.schema.LongSchema; -import org.apache.pulsar.client.impl.schema.ShortSchema; -import org.apache.pulsar.client.impl.schema.StringSchema; -import org.apache.pulsar.client.impl.schema.TimeSchema; -import org.apache.pulsar.client.impl.schema.TimestampSchema; -import org.apache.pulsar.common.api.raw.RawMessage; -import org.apache.pulsar.common.naming.TopicName; -import org.apache.pulsar.common.schema.SchemaInfo; -import org.apache.pulsar.common.schema.SchemaType; -import org.testng.Assert; -import org.testng.annotations.Test; - -import java.sql.Time; -import java.sql.Timestamp; -import java.util.Date; -import java.util.List; - -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; - -@Slf4j -public class TestPulsarPrimitiveSchemaHandler { - - private static final TopicName stringTopicName = TopicName.get("persistent", "tenant-1", "ns-1", "topic-1"); - @Test - public void testPulsarPrimitiveSchemaHandler() { - PulsarPrimitiveSchemaHandler pulsarPrimitiveSchemaHandler; - RawMessage rawMessage = mock(RawMessage.class); - SchemaInfo schemaInfoInt8 = SchemaInfo.builder().type(SchemaType.INT8).build(); - pulsarPrimitiveSchemaHandler = new PulsarPrimitiveSchemaHandler(schemaInfoInt8); - byte int8Value = 1; - when(rawMessage.getData()).thenReturn(ByteBufAllocator.DEFAULT.buffer().writeBytes(ByteSchema.of().encode(int8Value))); - Assert.assertEquals(int8Value, (byte)pulsarPrimitiveSchemaHandler.deserialize(rawMessage.getData())); - - SchemaInfo schemaInfoInt16 = SchemaInfo.builder().type(SchemaType.INT16).build(); - pulsarPrimitiveSchemaHandler = new PulsarPrimitiveSchemaHandler(schemaInfoInt16); - short int16Value = 2; - when(rawMessage.getData()).thenReturn(ByteBufAllocator.DEFAULT.buffer().writeBytes(ShortSchema.of().encode(int16Value))); - Assert.assertEquals(int16Value, pulsarPrimitiveSchemaHandler.deserialize(rawMessage.getData())); - - SchemaInfo schemaInfoInt32 = SchemaInfo.builder().type(SchemaType.INT32).build(); - pulsarPrimitiveSchemaHandler = new PulsarPrimitiveSchemaHandler(schemaInfoInt32); - int int32Value = 2; - when(rawMessage.getData()).thenReturn(ByteBufAllocator.DEFAULT.buffer().writeBytes(IntSchema.of().encode(int32Value))); - Assert.assertEquals(int32Value, pulsarPrimitiveSchemaHandler.deserialize(rawMessage.getData())); - - SchemaInfo schemaInfoInt64 = SchemaInfo.builder().type(SchemaType.INT64).build(); - pulsarPrimitiveSchemaHandler = new PulsarPrimitiveSchemaHandler(schemaInfoInt64); - long int64Value = 2; - when(rawMessage.getData()).thenReturn(ByteBufAllocator.DEFAULT.buffer().writeBytes(LongSchema.of().encode(int64Value))); - Assert.assertEquals(int64Value, pulsarPrimitiveSchemaHandler.deserialize(rawMessage.getData())); - - SchemaInfo schemaInfoString = SchemaInfo.builder().type(SchemaType.STRING).build(); - pulsarPrimitiveSchemaHandler = new PulsarPrimitiveSchemaHandler(schemaInfoString); - String stringValue = "test"; - when(rawMessage.getData()).thenReturn(ByteBufAllocator.DEFAULT.buffer().writeBytes(StringSchema.utf8().encode(stringValue))); - Assert.assertEquals(stringValue, pulsarPrimitiveSchemaHandler.deserialize(rawMessage.getData())); - - SchemaInfo schemaInfoFloat = SchemaInfo.builder().type(SchemaType.FLOAT).build(); - pulsarPrimitiveSchemaHandler = new PulsarPrimitiveSchemaHandler(schemaInfoFloat); - float floatValue = 0.2f; - when(rawMessage.getData()).thenReturn(ByteBufAllocator.DEFAULT.buffer().writeBytes(FloatSchema.of().encode(floatValue))); - Assert.assertEquals(floatValue, pulsarPrimitiveSchemaHandler.deserialize(rawMessage.getData())); - - SchemaInfo schemaInfoDouble = SchemaInfo.builder().type(SchemaType.DOUBLE).build(); - pulsarPrimitiveSchemaHandler = new PulsarPrimitiveSchemaHandler(schemaInfoDouble); - double doubleValue = 0.22d; - when(rawMessage.getData()).thenReturn(ByteBufAllocator.DEFAULT.buffer().writeBytes(DoubleSchema.of().encode(doubleValue))); - Assert.assertEquals(doubleValue, pulsarPrimitiveSchemaHandler.deserialize(rawMessage.getData())); - - SchemaInfo schemaInfoBoolean = SchemaInfo.builder().type(SchemaType.BOOLEAN).build(); - pulsarPrimitiveSchemaHandler = new PulsarPrimitiveSchemaHandler(schemaInfoBoolean); - boolean booleanValue = true; - when(rawMessage.getData()).thenReturn(ByteBufAllocator.DEFAULT.buffer().writeBytes(BooleanSchema.of().encode(booleanValue))); - Assert.assertEquals(booleanValue, pulsarPrimitiveSchemaHandler.deserialize(rawMessage.getData())); - - SchemaInfo schemaInfoBytes = SchemaInfo.builder().type(SchemaType.BYTES).build(); - pulsarPrimitiveSchemaHandler = new PulsarPrimitiveSchemaHandler(schemaInfoBytes); - byte[] bytesValue = new byte[1]; - bytesValue[0] = 1; - when(rawMessage.getData()).thenReturn(ByteBufAllocator.DEFAULT.buffer().writeBytes(BytesSchema.of().encode(bytesValue))); - Assert.assertEquals(bytesValue, pulsarPrimitiveSchemaHandler.deserialize(rawMessage.getData())); - - SchemaInfo schemaInfoDate = SchemaInfo.builder().type(SchemaType.DATE).build(); - pulsarPrimitiveSchemaHandler = new PulsarPrimitiveSchemaHandler(schemaInfoDate); - Date dateValue = new Date(System.currentTimeMillis()); - when(rawMessage.getData()).thenReturn(ByteBufAllocator.DEFAULT.buffer().writeBytes(DateSchema.of().encode(dateValue))); - Object dateDeserializeValue = pulsarPrimitiveSchemaHandler.deserialize(rawMessage.getData()); - Assert.assertEquals(dateValue.getTime(), dateDeserializeValue); - - SchemaInfo schemaInfoTime = SchemaInfo.builder().type(SchemaType.TIME).build(); - pulsarPrimitiveSchemaHandler = new PulsarPrimitiveSchemaHandler(schemaInfoTime); - Time timeValue = new Time(System.currentTimeMillis()); - when(rawMessage.getData()).thenReturn(ByteBufAllocator.DEFAULT.buffer().writeBytes(TimeSchema.of().encode(timeValue))); - Object timeDeserializeValue = pulsarPrimitiveSchemaHandler.deserialize(rawMessage.getData()); - Assert.assertEquals(timeValue.getTime(), timeDeserializeValue); - - SchemaInfo schemaInfoTimestamp = SchemaInfo.builder().type(SchemaType.TIMESTAMP).build(); - pulsarPrimitiveSchemaHandler = new PulsarPrimitiveSchemaHandler(schemaInfoTimestamp); - Timestamp timestampValue = new Timestamp(System.currentTimeMillis()); - when(rawMessage.getData()).thenReturn(ByteBufAllocator.DEFAULT.buffer().writeBytes(TimestampSchema.of().encode(timestampValue))); - Object timestampDeserializeValue = pulsarPrimitiveSchemaHandler.deserialize(rawMessage.getData()); - Assert.assertEquals(timestampValue.getTime(), timestampDeserializeValue); - } - - @Test - public void testNewPulsarPrimitiveSchemaHandler() { - RawMessage rawMessage = mock(RawMessage.class); - SchemaHandler schemaHandler = PulsarSchemaHandlers.newPulsarSchemaHandler( - null, - null, - StringSchema.utf8().getSchemaInfo(), - null); - - String stringValue = "test"; - when(rawMessage.getData()).thenReturn(ByteBufAllocator.DEFAULT.buffer().writeBytes(StringSchema.utf8().encode(stringValue))); - - Object deserializeValue = schemaHandler.deserialize(rawMessage.getData()); - Assert.assertEquals(stringValue, (String)deserializeValue); - Assert.assertEquals(stringValue, (String)deserializeValue); - - } - - @Test - public void testNewColumnMetadata() { - List columnMetadataList = PulsarMetadata.getPulsarColumns(stringTopicName, - StringSchema.utf8().getSchemaInfo(), false, null); - Assert.assertEquals(columnMetadataList.size(), 1); - ColumnMetadata columnMetadata = columnMetadataList.get(0); - Assert.assertEquals("__value__", columnMetadata.getName()); - Assert.assertEquals("varchar", columnMetadata.getType().toString()); - } -} diff --git a/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/TestPulsarRecordCursor.java b/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/TestPulsarRecordCursor.java index 1e28fb9..aadf0dc 100644 --- a/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/TestPulsarRecordCursor.java +++ b/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/TestPulsarRecordCursor.java @@ -18,27 +18,50 @@ */ package org.apache.pulsar.sql.presto; +import com.fasterxml.jackson.databind.ObjectMapper; import io.airlift.log.Logger; import io.netty.buffer.ByteBuf; -import io.netty.buffer.Unpooled; +import io.prestosql.spi.predicate.TupleDomain; +import io.prestosql.spi.type.RowType; +import lombok.Data; +import org.apache.bookkeeper.mledger.AsyncCallbacks; +import org.apache.bookkeeper.mledger.Entry; +import org.apache.bookkeeper.mledger.ManagedLedgerConfig; +import org.apache.bookkeeper.mledger.ManagedLedgerFactory; +import org.apache.bookkeeper.mledger.ReadOnlyCursor; +import org.apache.bookkeeper.mledger.impl.EntryImpl; +import org.apache.bookkeeper.mledger.impl.PositionImpl; +import org.apache.bookkeeper.mledger.impl.ReadOnlyCursorImpl; +import org.apache.bookkeeper.mledger.proto.MLDataFormats; +import org.apache.bookkeeper.stats.NullStatsProvider; +import org.apache.pulsar.client.api.Schema; +import org.apache.pulsar.client.impl.schema.KeyValueSchema; +import org.apache.pulsar.common.api.proto.MessageMetadata; import org.apache.pulsar.common.naming.TopicName; -import org.mockito.Mock; -import org.mockito.Mockito; +import org.apache.pulsar.common.protocol.Commands; +import org.apache.pulsar.common.schema.KeyValue; +import org.apache.pulsar.common.schema.KeyValueEncodingType; +import org.mockito.invocation.InvocationOnMock; +import org.mockito.stubbing.Answer; import org.testng.annotations.Test; -import java.lang.reflect.InvocationTargetException; -import java.lang.reflect.Method; -import java.nio.ByteBuffer; +import java.nio.charset.Charset; +import java.util.Arrays; +import java.util.HashMap; import java.util.LinkedList; import java.util.List; import java.util.Map; import static java.util.concurrent.CompletableFuture.completedFuture; +import static org.apache.pulsar.common.protocol.Commands.serializeMetadataAndPayload; +import static org.mockito.ArgumentMatchers.anyInt; import static org.mockito.Matchers.any; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.spy; import static org.mockito.Mockito.when; import static org.testng.Assert.assertEquals; -import static org.testng.Assert.assertFalse; import static org.testng.Assert.assertNotNull; import static org.testng.Assert.assertTrue; @@ -53,15 +76,13 @@ public void testTopics() throws Exception { log.info("!------ topic %s ------!", entry.getKey()); setup(); + + List fooColumnHandles = topicsToColumnHandles.get(entry.getKey()); PulsarRecordCursor pulsarRecordCursor = entry.getValue(); - SchemaHandler schemaHandler = pulsarRecordCursor.getSchemaHandler(); PulsarSqlSchemaInfoProvider pulsarSqlSchemaInfoProvider = mock(PulsarSqlSchemaInfoProvider.class); - if (schemaHandler instanceof AvroSchemaHandler) { - AvroSchemaHandler avroSchemaHandler = (AvroSchemaHandler) schemaHandler; - avroSchemaHandler.getSchema().setSchemaInfoProvider(pulsarSqlSchemaInfoProvider); - when(pulsarSqlSchemaInfoProvider.getSchemaByVersion(any())).thenReturn(completedFuture(avroSchemaHandler.getSchemaInfo())); - } + when(pulsarSqlSchemaInfoProvider.getSchemaByVersion(any())).thenReturn(completedFuture(topicsToSchemas.get(entry.getKey().getSchemaName()))); + pulsarRecordCursor.setPulsarSqlSchemaInfoProvider(pulsarSqlSchemaInfoProvider); TopicName topicName = entry.getKey(); @@ -99,40 +120,10 @@ public void testTopics() throws Exception { } else if (fooColumnHandles.get(i).getName().equals("date")) { pulsarRecordCursor.getLong(i); columnsSeen.add(fooColumnHandles.get(i).getName()); - } else if (fooColumnHandles.get(i).getName().equals("bar.field1")) { - assertEquals(pulsarRecordCursor.getLong(i), ((Integer) fooFunctions.get("bar.field1").apply(count)).longValue()); - columnsSeen.add(fooColumnHandles.get(i).getName()); - } else if (fooColumnHandles.get(i).getName().equals("bar.field2")) { - assertEquals(pulsarRecordCursor.getSlice(i).getBytes(), ((String) fooFunctions.get("bar.field2").apply(count)).getBytes()); - columnsSeen.add(fooColumnHandles.get(i).getName()); - } else if (fooColumnHandles.get(i).getName().equals("bar.field3")) { - assertEquals(pulsarRecordCursor.getLong(i), Float.floatToIntBits(((Float) fooFunctions.get("bar.field3").apply(count)).floatValue())); - columnsSeen.add(fooColumnHandles.get(i).getName()); - } else if (fooColumnHandles.get(i).getName().equals("bar.test.field4")) { - assertEquals(pulsarRecordCursor.getDouble(i), ((Double) fooFunctions.get("bar.test.field4").apply(count)).doubleValue()); - columnsSeen.add(fooColumnHandles.get(i).getName()); - } else if (fooColumnHandles.get(i).getName().equals("bar.test.field5")) { - assertEquals(pulsarRecordCursor.getBoolean(i), ((Boolean) fooFunctions.get("bar.test.field5").apply(count)).booleanValue()); - columnsSeen.add(fooColumnHandles.get(i).getName()); - } else if (fooColumnHandles.get(i).getName().equals("bar.test.field6")) { - assertEquals(pulsarRecordCursor.getLong(i), ((Long) fooFunctions.get("bar.test.field6").apply(count)).longValue()); - columnsSeen.add(fooColumnHandles.get(i).getName()); - } else if (fooColumnHandles.get(i).getName().equals("bar.test.foobar.field1")) { - assertEquals(pulsarRecordCursor.getLong(i), ((Integer) fooFunctions.get("bar.test.foobar.field1").apply(count)).longValue()); - columnsSeen.add(fooColumnHandles.get(i).getName()); - } else if (fooColumnHandles.get(i).getName().equals("bar.test2.field4")) { - assertEquals(pulsarRecordCursor.getDouble(i), ((Double) fooFunctions.get("bar.test2.field4").apply(count)).doubleValue()); - columnsSeen.add(fooColumnHandles.get(i).getName()); - } else if (fooColumnHandles.get(i).getName().equals("bar.test2.field5")) { - assertEquals(pulsarRecordCursor.getBoolean(i), ((Boolean) fooFunctions.get("bar.test2.field5").apply(count)).booleanValue()); - columnsSeen.add(fooColumnHandles.get(i).getName()); - } else if (fooColumnHandles.get(i).getName().equals("bar.test2.field6")) { - assertEquals(pulsarRecordCursor.getLong(i), ((Long) fooFunctions.get("bar.test2.field6").apply(count)).longValue()); - columnsSeen.add(fooColumnHandles.get(i).getName()); - } else if (fooColumnHandles.get(i).getName().equals("bar.test2.foobar.field1")) { - assertEquals(pulsarRecordCursor.getLong(i), ((Integer) fooFunctions.get("bar.test2.foobar.field1").apply(count)).longValue()); + } else if (fooColumnHandles.get(i).getName().equals("bar")) { + assertTrue(fooColumnHandles.get(i).getType() instanceof RowType); columnsSeen.add(fooColumnHandles.get(i).getName()); - } else if (fooColumnHandles.get(i).getName().equals("field7")) { + }else if (fooColumnHandles.get(i).getName().equals("field7")) { assertEquals(pulsarRecordCursor.getSlice(i).getBytes(), fooFunctions.get("field7").apply(count).toString().getBytes()); columnsSeen.add(fooColumnHandles.get(i).getName()); } else { @@ -152,44 +143,276 @@ public void testTopics() throws Exception { } } - @Test - public void testRecordToBytes() throws NoSuchMethodException, InvocationTargetException, IllegalAccessException { - PulsarRecordCursor pulsarRecordCursor = Mockito.mock(PulsarRecordCursor.class); - Method method = PulsarRecordCursor.class.getDeclaredMethod("toBytes", Object.class); - method.setAccessible(true); - - final String msg = "Hello!"; - - byte[] bytes = msg.getBytes(); - Object obj = method.invoke(pulsarRecordCursor, bytes); - assertNotNull(obj); - assertEquals(new String((byte[]) obj), msg); - - ByteBuffer byteBuffer1 = ByteBuffer.wrap(msg.getBytes()); - assertTrue(byteBuffer1.hasArray()); - obj = method.invoke(pulsarRecordCursor, byteBuffer1); - assertNotNull(obj); - assertEquals(new String((byte[]) obj), msg); - - ByteBuffer byteBuffer2 = ByteBuffer.allocateDirect(msg.getBytes().length); - byteBuffer2.put(msg.getBytes()); - assertFalse(byteBuffer2.hasArray()); - obj = method.invoke(pulsarRecordCursor, byteBuffer2); - assertNotNull(obj); - assertEquals(new String((byte[]) obj), msg); - - ByteBuf byteBuf1 = Unpooled.wrappedBuffer(msg.getBytes()); - assertTrue(byteBuf1.hasArray()); - obj = method.invoke(pulsarRecordCursor, byteBuf1); - assertNotNull(obj); - assertEquals(new String((byte[]) obj), msg); - - ByteBuf byteBuf2 = Unpooled.directBuffer(); - byteBuf2.writeBytes(msg.getBytes()); - assertFalse(byteBuf2.hasArray()); - obj = method.invoke(pulsarRecordCursor, byteBuf2); - assertNotNull(obj); - assertEquals(new String((byte[]) obj), msg); + @Test(singleThreaded = true) + public void TestKeyValueStructSchema() throws Exception { + + TopicName topicName = TopicName.get("persistent", NAMESPACE_NAME_1, "topic-4"); + Long entriesNum = 5L; + + for (KeyValueEncodingType encodingType : + Arrays.asList(KeyValueEncodingType.INLINE, KeyValueEncodingType.SEPARATED)) { + + KeyValueSchema schema = (KeyValueSchema) Schema.KeyValue(Schema.JSON(Foo.class), Schema.AVRO(Boo.class), + encodingType); + + Foo foo = new Foo(); + foo.field1 = "field1-value"; + foo.field2 = 20; + Boo boo = new Boo(); + boo.field1 = "field1-value"; + boo.field2 = true; + boo.field3 = 10.2; + + KeyValue message = new KeyValue<>(foo, boo); + List ColumnHandles = getColumnColumnHandles(topicName, schema.getSchemaInfo(), PulsarColumnHandle.HandleKeyValueType.NONE, true); + PulsarRecordCursor pulsarRecordCursor = mockKeyValueSchemaPulsarRecordCursor(entriesNum, topicName, + schema, message, ColumnHandles); + + assertNotNull(pulsarRecordCursor); + Long count = 0L; + while (pulsarRecordCursor.advanceNextPosition()) { + List columnsSeen = new LinkedList<>(); + for (int i = 0; i < ColumnHandles.size(); i++) { + if (pulsarRecordCursor.isNull(i)) { + columnsSeen.add(ColumnHandles.get(i).getName()); + } else { + if (ColumnHandles.get(i).getName().equals("field1")) { + assertEquals(pulsarRecordCursor.getSlice(i).getBytes(), boo.field1.getBytes()); + columnsSeen.add(ColumnHandles.get(i).getName()); + } else if (ColumnHandles.get(i).getName().equals("field2")) { + assertEquals(pulsarRecordCursor.getBoolean(i), boo.field2.booleanValue()); + columnsSeen.add(ColumnHandles.get(i).getName()); + } else if (ColumnHandles.get(i).getName().equals("field3")) { + assertEquals((Double) pulsarRecordCursor.getDouble(i), (Double) boo.field3); + columnsSeen.add(ColumnHandles.get(i).getName()); + } else if (ColumnHandles.get(i).getName().equals(PulsarColumnMetadata.KEY_SCHEMA_COLUMN_PREFIX + + "field1")) { + assertEquals(pulsarRecordCursor.getSlice(i).getBytes(), foo.field1.getBytes()); + columnsSeen.add(ColumnHandles.get(i).getName()); + } else if (ColumnHandles.get(i).getName().equals(PulsarColumnMetadata.KEY_SCHEMA_COLUMN_PREFIX + + "field2")) { + assertEquals(pulsarRecordCursor.getLong(i), Long.valueOf(foo.field2).longValue()); + columnsSeen.add(ColumnHandles.get(i).getName()); + } else { + if (PulsarInternalColumn.getInternalFieldsMap().containsKey(ColumnHandles.get(i).getName())) { + columnsSeen.add(ColumnHandles.get(i).getName()); + } + } + } + } + assertEquals(columnsSeen.size(), ColumnHandles.size()); + count++; + } + assertEquals(count, entriesNum); + pulsarRecordCursor.close(); + } + } + + @Test(singleThreaded = true) + public void TestKeyValuePrimitiveSchema() throws Exception { + + TopicName topicName = TopicName.get("persistent", NAMESPACE_NAME_1, "topic-4"); + Long entriesNum = 5L; + + for (KeyValueEncodingType encodingType : + Arrays.asList(KeyValueEncodingType.INLINE, KeyValueEncodingType.SEPARATED)) { + + KeyValueSchema schema = (KeyValueSchema) Schema.KeyValue(Schema.INT32, Schema.STRING, + encodingType); + + String value = "primitive_message_value"; + Integer key = 23; + KeyValue message = new KeyValue<>(key, value); + + List ColumnHandles = getColumnColumnHandles(topicName, schema.getSchemaInfo(), PulsarColumnHandle.HandleKeyValueType.NONE, true); + PulsarRecordCursor pulsarRecordCursor = mockKeyValueSchemaPulsarRecordCursor(entriesNum, topicName, + schema, message, ColumnHandles); + + assertNotNull(pulsarRecordCursor); + Long count = 0L; + while (pulsarRecordCursor.advanceNextPosition()) { + List columnsSeen = new LinkedList<>(); + for (int i = 0; i < ColumnHandles.size(); i++) { + if (pulsarRecordCursor.isNull(i)) { + columnsSeen.add(ColumnHandles.get(i).getName()); + } else { + if (ColumnHandles.get(i).getName().equals(PRIMITIVE_COLUMN_NAME)) { + assertEquals(pulsarRecordCursor.getSlice(i).getBytes(), value.getBytes()); + columnsSeen.add(ColumnHandles.get(i).getName()); + } else if (ColumnHandles.get(i).getName().equals(KEY_SCHEMA_COLUMN_PREFIX + + PRIMITIVE_COLUMN_NAME)) { + assertEquals((Long) pulsarRecordCursor.getLong(i), Long.valueOf(key)); + columnsSeen.add(ColumnHandles.get(i).getName()); + } else { + if (PulsarInternalColumn.getInternalFieldsMap().containsKey(ColumnHandles.get(i).getName())) { + columnsSeen.add(ColumnHandles.get(i).getName()); + } + } + } + } + assertEquals(columnsSeen.size(), ColumnHandles.size()); + count++; + } + assertEquals(count, entriesNum); + pulsarRecordCursor.close(); + } + } + + + /** + * mock a simple PulsarRecordCursor for KeyValueSchema test. + * @param entriesNum + * @param topicName + * @param schema + * @param message + * @param ColumnHandles + * @return + * @throws Exception + */ + private PulsarRecordCursor mockKeyValueSchemaPulsarRecordCursor(final Long entriesNum, final TopicName topicName, + final KeyValueSchema schema, KeyValue message, List ColumnHandles) throws Exception { + + ManagedLedgerFactory managedLedgerFactory = mock(ManagedLedgerFactory.class); + + when(managedLedgerFactory.openReadOnlyCursor(any(), any(), any())).then(new Answer() { + + private Map positions = new HashMap<>(); + + @Override + public ReadOnlyCursor answer(InvocationOnMock invocationOnMock) throws Throwable { + Object[] args = invocationOnMock.getArguments(); + String topic = (String) args[0]; + PositionImpl positionImpl = (PositionImpl) args[1]; + int position = positionImpl.getEntryId() == -1 ? 0 : (int) positionImpl.getEntryId(); + + positions.put(topic, position); + ReadOnlyCursorImpl readOnlyCursor = mock(ReadOnlyCursorImpl.class); + doReturn(entriesNum).when(readOnlyCursor).getNumberOfEntries(); + + doAnswer(new Answer() { + @Override + public Void answer(InvocationOnMock invocation) throws Throwable { + Object[] args = invocation.getArguments(); + Integer skipEntries = (Integer) args[0]; + positions.put(topic, positions.get(topic) + skipEntries); + return null; + } + }).when(readOnlyCursor).skipEntries(anyInt()); + + when(readOnlyCursor.getReadPosition()).thenAnswer(new Answer() { + @Override + public PositionImpl answer(InvocationOnMock invocationOnMock) throws Throwable { + return PositionImpl.get(0, positions.get(topic)); + } + }); + + doAnswer(new Answer() { + @Override + public Object answer(InvocationOnMock invocationOnMock) throws Throwable { + Object[] args = invocationOnMock.getArguments(); + Integer readEntries = (Integer) args[0]; + AsyncCallbacks.ReadEntriesCallback callback = (AsyncCallbacks.ReadEntriesCallback) args[1]; + Object ctx = args[2]; + + new Thread(new Runnable() { + @Override + public void run() { + List entries = new LinkedList<>(); + for (int i = 0; i < readEntries; i++) { + + MessageMetadata messageMetadata = + new MessageMetadata() + .setProducerName("test-producer").setSequenceId(positions.get(topic)) + .setPublishTime(System.currentTimeMillis()); + + if (KeyValueEncodingType.SEPARATED.equals(schema.getKeyValueEncodingType())) { + messageMetadata + .setPartitionKey(new String(schema + .getKeySchema().encode(message.getKey()), Charset.forName( + "UTF-8"))) + .setPartitionKeyB64Encoded(false); + } + + ByteBuf dataPayload = io.netty.buffer.Unpooled + .copiedBuffer(schema.encode(message)); + + ByteBuf byteBuf = serializeMetadataAndPayload( + Commands.ChecksumType.Crc32c, messageMetadata, dataPayload); + + entries.add(EntryImpl.create(0, positions.get(topic), byteBuf)); + positions.put(topic, positions.get(topic) + 1); + } + + callback.readEntriesComplete(entries, ctx); + } + }).start(); + + return null; + } + }).when(readOnlyCursor).asyncReadEntries(anyInt(), any(), any(), any()); + + when(readOnlyCursor.hasMoreEntries()).thenAnswer(new Answer() { + @Override + public Boolean answer(InvocationOnMock invocationOnMock) throws Throwable { + return positions.get(topic) < entriesNum; + } + }); + + when(readOnlyCursor.getNumberOfEntries(any())).then(new Answer() { + @Override + public Long answer(InvocationOnMock invocationOnMock) throws Throwable { + Object[] args = invocationOnMock.getArguments(); + com.google.common.collect.Range range + = (com.google.common.collect.Range) args[0]; + return (range.upperEndpoint().getEntryId() + 1) - range.lowerEndpoint().getEntryId(); + } + }); + + when(readOnlyCursor.getCurrentLedgerInfo()).thenReturn(MLDataFormats.ManagedLedgerInfo.LedgerInfo.newBuilder().setLedgerId(0).build()); + + return readOnlyCursor; + } + }); + + ObjectMapper objectMapper = new ObjectMapper(); + + PulsarSplit split = new PulsarSplit(0, pulsarConnectorId.toString(), + topicName.getNamespace(), topicName.getLocalName(), topicName.getLocalName(), + entriesNum, + new String(schema.getSchemaInfo().getSchema()), + schema.getSchemaInfo().getType(), + 0, entriesNum, + 0, 0, TupleDomain.all(), + objectMapper.writeValueAsString( + schema.getSchemaInfo().getProperties()), null); + + PulsarRecordCursor pulsarRecordCursor = spy(new PulsarRecordCursor( + ColumnHandles, split, + pulsarConnectorConfig, managedLedgerFactory, new ManagedLedgerConfig(), + new PulsarConnectorMetricsTracker(new NullStatsProvider()), dispatchingRowDecoderFactory)); + + PulsarSqlSchemaInfoProvider pulsarSqlSchemaInfoProvider = mock(PulsarSqlSchemaInfoProvider.class); + when(pulsarSqlSchemaInfoProvider.getSchemaByVersion(any())).thenReturn(completedFuture(schema.getSchemaInfo())); + pulsarRecordCursor.setPulsarSqlSchemaInfoProvider(pulsarSqlSchemaInfoProvider); + + return pulsarRecordCursor; + } + + + final static String KEY_SCHEMA_COLUMN_PREFIX = "__key."; + final static String PRIMITIVE_COLUMN_NAME = "__value__"; + + @Data + static class Foo { + private String field1; + private Integer field2; + } + + @Data + static class Boo { + private String field1; + private Boolean field2; + private Double field3; } } diff --git a/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/decoder/AbstractDecoderTester.java b/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/decoder/AbstractDecoderTester.java new file mode 100644 index 0000000..cd4fcaf --- /dev/null +++ b/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/decoder/AbstractDecoderTester.java @@ -0,0 +1,133 @@ +/** + * 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.decoder; + +import io.airlift.slice.Slice; +import io.prestosql.decoder.DecoderColumnHandle; +import io.prestosql.decoder.FieldValueProvider; +import io.prestosql.spi.block.Block; +import io.prestosql.spi.connector.ColumnMetadata; +import io.prestosql.spi.connector.ConnectorContext; +import io.prestosql.spi.type.Type; +import io.prestosql.testing.TestingConnectorContext; +import org.apache.pulsar.common.naming.NamespaceName; +import org.apache.pulsar.common.naming.TopicName; +import org.apache.pulsar.common.schema.SchemaInfo; +import org.apache.pulsar.sql.presto.PulsarColumnHandle; +import org.apache.pulsar.sql.presto.PulsarColumnMetadata; +import org.apache.pulsar.sql.presto.PulsarConnectorConfig; +import org.apache.pulsar.sql.presto.PulsarConnectorId; +import org.apache.pulsar.sql.presto.PulsarDispatchingRowDecoderFactory; +import org.apache.pulsar.sql.presto.PulsarMetadata; +import org.apache.pulsar.sql.presto.PulsarRowDecoder; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +import static org.mockito.Mockito.spy; +import static org.testng.Assert.assertNotNull; + +/** + * Abstract superclass for TestXXDecoder (e.g. TestAvroDecoder 、TestJsonDecoder). + */ +public abstract class AbstractDecoderTester { + + protected PulsarDispatchingRowDecoderFactory decoderFactory; + protected PulsarConnectorId pulsarConnectorId = new PulsarConnectorId("test-connector"); + protected SchemaInfo schemaInfo; + protected TopicName topicName; + protected List pulsarColumnHandle; + protected PulsarRowDecoder pulsarRowDecoder; + protected DecoderTestUtil decoderTestUtil; + protected PulsarConnectorConfig pulsarConnectorConfig; + protected PulsarMetadata pulsarMetadata; + + protected void init() { + ConnectorContext prestoConnectorContext = new TestingConnectorContext(); + this.decoderFactory = new PulsarDispatchingRowDecoderFactory(prestoConnectorContext.getTypeManager()); + this.pulsarConnectorConfig = spy(new PulsarConnectorConfig()); + this.pulsarConnectorConfig.setMaxEntryReadBatchSize(1); + this.pulsarConnectorConfig.setMaxSplitEntryQueueSize(10); + this.pulsarConnectorConfig.setMaxSplitMessageQueueSize(100); + this.pulsarMetadata = new PulsarMetadata(pulsarConnectorId, this.pulsarConnectorConfig, decoderFactory); + this.topicName = TopicName.get("persistent", NamespaceName.get("tenant-1", "ns-1"), "topic-1"); + } + + protected void checkArrayValues(Block block, Type type, Object value) { + decoderTestUtil.checkArrayValues(block, type, value); + } + + protected void checkMapValues(Block block, Type type, Object value) { + decoderTestUtil.checkMapValues(block, type, value); + } + + protected void checkRowValues(Block block, Type type, Object value) { + decoderTestUtil.checkRowValues(block, type, value); + } + + protected void checkValue(Map decodedRow, DecoderColumnHandle handle, Slice value) { + decoderTestUtil.checkValue(decodedRow, handle, value); + } + + protected void checkValue(Map decodedRow, DecoderColumnHandle handle, String value) { + decoderTestUtil.checkValue(decodedRow, handle, value); + } + + protected void checkValue(Map decodedRow, DecoderColumnHandle handle, long value) { + decoderTestUtil.checkValue(decodedRow, handle, value); + } + + protected void checkValue(Map decodedRow, DecoderColumnHandle handle, double value) { + decoderTestUtil.checkValue(decodedRow, handle, value); + } + + protected void checkValue(Map decodedRow, DecoderColumnHandle handle, boolean value) { + decoderTestUtil.checkValue(decodedRow, handle, value); + } + + protected Block getBlock(Map decodedRow, DecoderColumnHandle handle) { + FieldValueProvider provider = decodedRow.get(handle); + assertNotNull(provider); + return provider.getBlock(); + } + + protected List getColumnColumnHandles(TopicName topicName, SchemaInfo schemaInfo, + PulsarColumnHandle.HandleKeyValueType handleKeyValueType, boolean includeInternalColumn, PulsarDispatchingRowDecoderFactory dispatchingRowDecoderFactory) { + List columnHandles = new ArrayList<>(); + List columnMetadata = pulsarMetadata.getPulsarColumns(topicName, schemaInfo, + includeInternalColumn, handleKeyValueType); + + columnMetadata.forEach(column -> { + PulsarColumnMetadata pulsarColumnMetadata = (PulsarColumnMetadata) column; + columnHandles.add(new PulsarColumnHandle( + pulsarConnectorId.toString(), + pulsarColumnMetadata.getNameWithCase(), + pulsarColumnMetadata.getType(), + pulsarColumnMetadata.isHidden(), + pulsarColumnMetadata.isInternal(), + pulsarColumnMetadata.getDecoderExtraInfo().getMapping(), + pulsarColumnMetadata.getDecoderExtraInfo().getDataFormat(), pulsarColumnMetadata.getDecoderExtraInfo().getFormatHint(), + pulsarColumnMetadata.getHandleKeyValueType())); + + }); + return columnHandles; + } + +} diff --git a/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/decoder/DecoderTestMessage.java b/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/decoder/DecoderTestMessage.java new file mode 100644 index 0000000..b4db048 --- /dev/null +++ b/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/decoder/DecoderTestMessage.java @@ -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.decoder; + +import lombok.Data; + +import java.util.List; +import java.util.Map; + +public class DecoderTestMessage { + + public static enum TestEnum { + TEST_ENUM_1, + TEST_ENUM_2, + TEST_ENUM_3 + } + + public int intField; + public String stringField; + public float floatField; + public double doubleField; + public boolean booleanField; + public long longField; + @org.apache.avro.reflect.AvroSchema("{ \"type\": \"long\", \"logicalType\": \"timestamp-millis\" }") + public long timestampField; + @org.apache.avro.reflect.AvroSchema("{ \"type\": \"int\", \"logicalType\": \"time-millis\" }") + public int timeField; + @org.apache.avro.reflect.AvroSchema("{ \"type\": \"int\", \"logicalType\": \"date\" }") + public int dateField; + public TestRow rowField; + public TestEnum enumField; + + public List arrayField; + public Map mapField; + public CompositeRow compositeRow; + + public static class TestRow { + public String stringField; + public int intField; + public NestedRow nestedRow; + } + + + public static class NestedRow { + public String stringField; + public long longField; + } + + + public static class CompositeRow { + public String stringField; + public List arrayField; + public Map mapField; + public NestedRow nestedRow; + public Map> structedField; + } + + /** + * POJO for cyclic detect. + */ + @Data + static public class CyclicFoo { + private String field1; + private Integer field2; + private CyclicBoo boo; + } + + @Data + static public class CyclicBoo { + private String field1; + private Boolean field2; + private CyclicFoo foo; + } + +} diff --git a/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/decoder/DecoderTestUtil.java b/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/decoder/DecoderTestUtil.java new file mode 100644 index 0000000..4c3c4a6 --- /dev/null +++ b/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/decoder/DecoderTestUtil.java @@ -0,0 +1,121 @@ +/** + * 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.decoder; + +import io.airlift.slice.Slice; +import io.prestosql.decoder.DecoderColumnHandle; +import io.prestosql.decoder.FieldValueProvider; +import io.prestosql.spi.block.Block; +import io.prestosql.spi.type.ArrayType; +import io.prestosql.spi.type.MapType; +import io.prestosql.spi.type.RowType; +import io.prestosql.spi.type.Type; +import java.util.Map; + +import static io.prestosql.testing.TestingConnectorSession.SESSION; +import static org.testng.Assert.*; + +/** + * Abstract util superclass for XXDecoderTestUtil (e.g. AvroDecoderTestUtil 、JsonDecoderTestUtil) + */ +public abstract class DecoderTestUtil { + + protected DecoderTestUtil() { + + } + + public abstract void checkArrayValues(Block block, Type type, Object value); + + public abstract void checkMapValues(Block block, Type type, Object value); + + public abstract void checkRowValues(Block block, Type type, Object value); + + public abstract void checkPrimitiveValue(Object actual, Object expected); + + public void checkField(Block actualBlock, Type type, int position, Object expectedValue) { + assertNotNull(type, "Type is null"); + assertNotNull(actualBlock, "actualBlock is null"); + assertTrue(!actualBlock.isNull(position)); + assertNotNull(expectedValue, "expectedValue is null"); + + if (type instanceof ArrayType) { + checkArrayValues(actualBlock.getObject(position, Block.class), type, expectedValue); + } else if (type instanceof MapType) { + checkMapValues(actualBlock.getObject(position, Block.class), type, expectedValue); + } else if (type instanceof RowType) { + checkRowValues(actualBlock.getObject(position, Block.class), type, expectedValue); + } else { + checkPrimitiveValue(getObjectValue(type, actualBlock, position), expectedValue); + } + } + + public boolean isIntegralType(Object value) { + return value instanceof Long + || value instanceof Integer + || value instanceof Short + || value instanceof Byte; + } + + public boolean isRealType(Object value) { + return value instanceof Float || value instanceof Double; + } + + public Object getObjectValue(Type type, Block block, int position) { + if (block.isNull(position)) { + return null; + } + return type.getObjectValue(SESSION, block, position); + } + + public void checkValue(Map decodedRow, DecoderColumnHandle handle, Slice value) { + FieldValueProvider provider = decodedRow.get(handle); + assertNotNull(provider); + assertEquals(provider.getSlice(), value); + } + + public void checkValue(Map decodedRow, DecoderColumnHandle handle, String value) { + FieldValueProvider provider = decodedRow.get(handle); + assertNotNull(provider); + assertEquals(provider.getSlice().toStringUtf8(), value); + } + + public void checkValue(Map decodedRow, DecoderColumnHandle handle, long value) { + FieldValueProvider provider = decodedRow.get(handle); + assertNotNull(provider); + assertEquals(provider.getLong(), value); + } + + public void checkValue(Map decodedRow, DecoderColumnHandle handle, double value) { + FieldValueProvider provider = decodedRow.get(handle); + assertNotNull(provider); + assertEquals(provider.getDouble(), value, 0.0001); + } + + public void checkValue(Map decodedRow, DecoderColumnHandle handle, boolean value) { + FieldValueProvider provider = decodedRow.get(handle); + assertNotNull(provider); + assertEquals(provider.getBoolean(), value); + } + + public void checkIsNull(Map decodedRow, DecoderColumnHandle handle) { + FieldValueProvider provider = decodedRow.get(handle); + assertNotNull(provider); + assertTrue(provider.isNull()); + } +} diff --git a/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/decoder/avro/AvroDecoderTestUtil.java b/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/decoder/avro/AvroDecoderTestUtil.java new file mode 100644 index 0000000..521807f --- /dev/null +++ b/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/decoder/avro/AvroDecoderTestUtil.java @@ -0,0 +1,199 @@ +/** + * 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.decoder.avro; + +import io.prestosql.spi.block.Block; +import io.prestosql.spi.type.*; +import org.apache.avro.generic.GenericEnumSymbol; +import org.apache.avro.generic.GenericFixed; +import org.apache.avro.generic.GenericRecord; +import org.apache.pulsar.sql.presto.decoder.DecoderTestUtil; + +import java.nio.ByteBuffer; +import java.util.List; +import java.util.Map; + +import static io.prestosql.spi.type.VarcharType.VARCHAR; +import static java.lang.String.format; +import static org.testng.Assert.*; + +/** + * TestUtil for AvroDecoder + */ +public class AvroDecoderTestUtil extends DecoderTestUtil { + public AvroDecoderTestUtil() { + super(); + } + + public void checkPrimitiveValue(Object actual, Object expected) { + if (actual == null || expected == null) { + assertNull(expected); + assertNull(actual); + } else if (actual instanceof CharSequence) { + assertTrue(expected instanceof CharSequence || expected instanceof GenericEnumSymbol); + assertEquals(actual.toString(), expected.toString()); + } else if (actual instanceof SqlVarbinary) { + if (expected instanceof GenericFixed) { + assertEquals(((SqlVarbinary) actual).getBytes(), ((GenericFixed) expected).bytes()); + } else if (expected instanceof ByteBuffer) { + assertEquals(((SqlVarbinary) actual).getBytes(), ((ByteBuffer) expected).array()); + } else { + fail(format("Unexpected value type %s", actual.getClass())); + } + } else if (isIntegralType(actual) && isIntegralType(expected)) { + assertEquals(((Number) actual).longValue(), ((Number) expected).longValue()); + } else if (isRealType(actual) && isRealType(expected)) { + assertEquals(((Number) actual).doubleValue(), ((Number) expected).doubleValue()); + } else { + assertEquals(actual, expected); + } + } + + + public void checkArrayValues(Block block, Type type, Object value) { + assertNotNull(type, "Type is null"); + assertTrue(type instanceof ArrayType, "Unexpected type"); + assertNotNull(block, "Block is null"); + assertNotNull(value, "Value is null"); + + List list = (List) value; + + assertEquals(block.getPositionCount(), list.size()); + Type elementType = ((ArrayType) type).getElementType(); + if (elementType instanceof ArrayType) { + for (int index = 0; index < block.getPositionCount(); index++) { + if (block.isNull(index)) { + assertNull(list.get(index)); + continue; + } + Block arrayBlock = block.getObject(index, Block.class); + checkArrayValues(arrayBlock, elementType, list.get(index)); + } + } else if (elementType instanceof MapType) { + for (int index = 0; index < block.getPositionCount(); index++) { + if (block.isNull(index)) { + assertNull(list.get(index)); + continue; + } + Block mapBlock = block.getObject(index, Block.class); + checkMapValues(mapBlock, elementType, list.get(index)); + } + } else if (elementType instanceof RowType) { + for (int index = 0; index < block.getPositionCount(); index++) { + if (block.isNull(index)) { + assertNull(list.get(index)); + continue; + } + Block rowBlock = block.getObject(index, Block.class); + checkRowValues(rowBlock, elementType, list.get(index)); + } + } else { + for (int index = 0; index < block.getPositionCount(); index++) { + checkPrimitiveValue(getObjectValue(elementType, block, index), list.get(index)); + } + } + } + + /** + * fix key as org.apache.avro.util.Utf8 + * + * @param block + * @param type + * @param value + */ + public void checkMapValues(Block block, Type type, Object value) { + assertNotNull(type, "Type is null"); + assertTrue(type instanceof MapType, "Unexpected type"); + assertTrue(((MapType) type).getKeyType() instanceof VarcharType, "Unexpected key type"); + assertNotNull(block, "Block is null"); + assertNotNull(value, "Value is null"); + + + Map expected = (Map) value; + + assertEquals(block.getPositionCount(), expected.size() * 2); + Type valueType = ((MapType) type).getValueType(); + if (valueType instanceof ArrayType) { + for (int index = 0; index < block.getPositionCount(); index += 2) { + String actualKey = VARCHAR.getSlice(block, index).toStringUtf8(); + assertTrue(expected.keySet().stream().anyMatch(e -> e.toString().equals(actualKey))); + if (block.isNull(index + 1)) { + assertNull(expected.get(actualKey)); + continue; + } + Block arrayBlock = block.getObject(index + 1, Block.class); + Object keyValue = expected.entrySet().stream().filter(e -> e.getKey().toString().equals(actualKey)).findFirst().get().getValue(); + checkArrayValues(arrayBlock, valueType, keyValue); + } + } else if (valueType instanceof MapType) { + for (int index = 0; index < block.getPositionCount(); index += 2) { + String actualKey = VARCHAR.getSlice(block, index).toStringUtf8(); + assertTrue(expected.keySet().stream().anyMatch(e -> e.toString().equals(actualKey))); + if (block.isNull(index + 1)) { + assertNull(expected.get(actualKey)); + continue; + } + Block mapBlock = block.getObject(index + 1, Block.class); + Object keyValue = expected.entrySet().stream().filter(e -> e.getKey().toString().equals(actualKey)).findFirst().get().getValue(); + checkMapValues(mapBlock, valueType, keyValue); + } + } else if (valueType instanceof RowType) { + for (int index = 0; index < block.getPositionCount(); index += 2) { + String actualKey = VARCHAR.getSlice(block, index).toStringUtf8(); + assertTrue(expected.keySet().stream().anyMatch(e -> e.toString().equals(actualKey))); + if (block.isNull(index + 1)) { + assertNull(expected.get(actualKey)); + continue; + } + Block rowBlock = block.getObject(index + 1, Block.class); + Object keyValue = expected.entrySet().stream().filter(e -> e.getKey().toString().equals(actualKey)).findFirst().get().getValue(); + checkRowValues(rowBlock, valueType, keyValue); + } + } else { + for (int index = 0; index < block.getPositionCount(); index += 2) { + String actualKey = VARCHAR.getSlice(block, index).toStringUtf8(); + assertTrue(expected.keySet().stream().anyMatch(e -> e.toString().equals(actualKey))); + Object keyValue = expected.entrySet().stream().filter(e -> e.getKey().toString().equals(actualKey)).findFirst().get().getValue(); + checkPrimitiveValue(getObjectValue(valueType, block, index + 1), keyValue); + } + } + } + + public void checkRowValues(Block block, Type type, Object value) { + assertNotNull(type, "Type is null"); + assertTrue(type instanceof RowType, "Unexpected type"); + assertNotNull(block, "Block is null"); + assertNotNull(value, "Value is null"); + + GenericRecord record = (GenericRecord) value; + RowType rowType = (RowType) type; + assertEquals(record.getSchema().getFields().size(), rowType.getFields().size(), "Avro field size mismatch"); + assertEquals(block.getPositionCount(), rowType.getFields().size(), "Presto type field size mismatch"); + for (int fieldIndex = 0; fieldIndex < rowType.getFields().size(); fieldIndex++) { + RowType.Field rowField = rowType.getFields().get(fieldIndex); + Object expectedValue = record.get(rowField.getName().get()); + if (block.isNull(fieldIndex)) { + assertNull(expectedValue); + continue; + } + checkField(block, rowField.getType(), fieldIndex, expectedValue); + } + } + +} diff --git a/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/decoder/avro/TestAvroDecoder.java b/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/decoder/avro/TestAvroDecoder.java new file mode 100644 index 0000000..39295a9 --- /dev/null +++ b/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/decoder/avro/TestAvroDecoder.java @@ -0,0 +1,290 @@ +/** + * 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.decoder.avro; + +import com.google.common.collect.ImmutableList; +import io.netty.buffer.ByteBuf; +import io.prestosql.decoder.DecoderColumnHandle; +import io.prestosql.decoder.FieldValueProvider; +import io.prestosql.spi.PrestoException; +import io.prestosql.spi.type.*; +import org.apache.pulsar.client.impl.schema.AvroSchema; +import org.apache.pulsar.client.impl.schema.generic.GenericAvroRecord; +import org.apache.pulsar.client.impl.schema.generic.GenericAvroSchema; +import org.apache.pulsar.sql.presto.PulsarColumnHandle; +import org.apache.pulsar.sql.presto.decoder.AbstractDecoderTester; +import org.apache.pulsar.sql.presto.decoder.DecoderTestMessage; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Test; + +import java.time.LocalDate; +import java.time.LocalTime; +import java.time.ZoneId; +import java.time.temporal.ChronoUnit; +import java.util.*; + +import static io.prestosql.spi.type.BigintType.BIGINT; +import static io.prestosql.spi.type.BooleanType.BOOLEAN; +import static io.prestosql.spi.type.DoubleType.DOUBLE; +import static io.prestosql.spi.type.IntegerType.INTEGER; +import static io.prestosql.spi.type.RealType.REAL; +import static io.prestosql.spi.type.VarcharType.VARCHAR; +import static java.lang.Float.floatToIntBits; +import static org.apache.pulsar.sql.presto.TestPulsarConnector.getPulsarConnectorId; +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertTrue; +import static org.testng.Assert.expectThrows; + +public class TestAvroDecoder extends AbstractDecoderTester { + + private AvroSchema schema; + + @BeforeMethod + public void init() { + super.init(); + schema = AvroSchema.of(DecoderTestMessage.class); + schemaInfo = schema.getSchemaInfo(); + pulsarColumnHandle = getColumnColumnHandles(topicName, schemaInfo, PulsarColumnHandle.HandleKeyValueType.NONE, false, decoderFactory); + pulsarRowDecoder = decoderFactory.createRowDecoder(topicName, schemaInfo, new HashSet<>(pulsarColumnHandle)); + decoderTestUtil = new AvroDecoderTestUtil(); + assertTrue(pulsarRowDecoder instanceof PulsarAvroRowDecoder); + } + + @Test + public void testPrimitiveType() { + DecoderTestMessage message = new DecoderTestMessage(); + message.stringField = "message_1"; + message.intField = 22; + message.floatField = 2.2f; + message.doubleField = 22.20D; + message.booleanField = true; + message.longField = 222L; + message.timestampField = System.currentTimeMillis(); + message.enumField = DecoderTestMessage.TestEnum.TEST_ENUM_1; + + LocalTime now = LocalTime.now(ZoneId.systemDefault()); + message.timeField = now.toSecondOfDay() * 1000; + + LocalDate localDate = LocalDate.now(); + LocalDate epoch = LocalDate.ofEpochDay(0); + message.dateField = Math.toIntExact(ChronoUnit.DAYS.between(epoch, localDate)); + + ByteBuf payload = io.netty.buffer.Unpooled + .copiedBuffer(schema.encode(message)); + Map decodedRow = pulsarRowDecoder.decodeRow(payload).get(); + + PulsarColumnHandle stringFieldColumnHandle = new PulsarColumnHandle(getPulsarConnectorId().toString(), + "stringField", VARCHAR, false, false, "stringField", null, null, PulsarColumnHandle.HandleKeyValueType.NONE); + checkValue(decodedRow, stringFieldColumnHandle, message.stringField); + + PulsarColumnHandle intFieldColumnHandle = new PulsarColumnHandle(getPulsarConnectorId().toString(), + "intField", INTEGER, false, false, "intField", null, null, PulsarColumnHandle.HandleKeyValueType.NONE); + checkValue(decodedRow, intFieldColumnHandle, message.intField); + + PulsarColumnHandle floatFieldColumnHandle = new PulsarColumnHandle(getPulsarConnectorId().toString(), + "floatField", REAL, false, false, "floatField", null, null, PulsarColumnHandle.HandleKeyValueType.NONE); + checkValue(decodedRow, floatFieldColumnHandle, floatToIntBits(message.floatField)); + + PulsarColumnHandle doubleFieldColumnHandle = new PulsarColumnHandle(getPulsarConnectorId().toString(), + "doubleField", DOUBLE, false, false, "doubleField", null, null, PulsarColumnHandle.HandleKeyValueType.NONE); + checkValue(decodedRow, doubleFieldColumnHandle, message.doubleField); + + PulsarColumnHandle booleanFieldColumnHandle = new PulsarColumnHandle(getPulsarConnectorId().toString(), + "booleanField", BOOLEAN, false, false, "booleanField", null, null, PulsarColumnHandle.HandleKeyValueType.NONE); + checkValue(decodedRow, booleanFieldColumnHandle, message.booleanField); + + PulsarColumnHandle longFieldColumnHandle = new PulsarColumnHandle(getPulsarConnectorId().toString(), + "longField", BIGINT, false, false, "longField", null, null, PulsarColumnHandle.HandleKeyValueType.NONE); + checkValue(decodedRow, longFieldColumnHandle, message.longField); + + PulsarColumnHandle enumFieldColumnHandle = new PulsarColumnHandle(getPulsarConnectorId().toString(), + "enumField", VARCHAR, false, false, "enumField", null, null, PulsarColumnHandle.HandleKeyValueType.NONE); + checkValue(decodedRow, enumFieldColumnHandle, message.enumField.toString()); + + } + + @Test + public void testRow() { + DecoderTestMessage message = new DecoderTestMessage(); + message.stringField = "message_2"; + DecoderTestMessage.TestRow testRow = new DecoderTestMessage.TestRow(); + message.rowField = testRow; + testRow.intField = 22; + testRow.stringField = "message_2_testRow"; + DecoderTestMessage.NestedRow nestedRow = new DecoderTestMessage.NestedRow(); + nestedRow.longField = 222L; + nestedRow.stringField = "message_2_nestedRow"; + testRow.nestedRow = nestedRow; + + byte[] bytes = schema.encode(message); + ByteBuf payload = io.netty.buffer.Unpooled + .copiedBuffer(bytes); + GenericAvroRecord genericRecord = (GenericAvroRecord) GenericAvroSchema.of(schemaInfo).decode(bytes); + Object fieldValue = genericRecord.getAvroRecord().get("rowField"); + + Map decodedRow = pulsarRowDecoder.decodeRow(payload).get(); + RowType columnType = RowType.from(ImmutableList.builder() + .add(RowType.field("stringField", VARCHAR)) + .add(RowType.field("intField", INTEGER)) + .add(RowType.field("nestedRow", RowType.from(ImmutableList.builder() + .add(RowType.field("stringField", VARCHAR)) + .add(RowType.field("longField", BIGINT)) + .build()))) + .build()); + + PulsarColumnHandle columnHandle = new PulsarColumnHandle(getPulsarConnectorId().toString(), + "rowField", columnType, false, false, "rowField", null, null, PulsarColumnHandle.HandleKeyValueType.NONE); + + checkRowValues(getBlock(decodedRow, columnHandle), columnHandle.getType(), fieldValue); + } + + @Test + public void testArray() { + DecoderTestMessage message = new DecoderTestMessage(); + message.arrayField = Arrays.asList("message_1", "message_2", "message_3"); + + byte[] bytes = schema.encode(message); + ByteBuf payload = io.netty.buffer.Unpooled + .copiedBuffer(bytes); + GenericAvroRecord genericRecord = (GenericAvroRecord) GenericAvroSchema.of(schemaInfo).decode(bytes); + Object fieldValue = genericRecord.getAvroRecord().get("arrayField"); + Map decodedRow = pulsarRowDecoder.decodeRow(payload).get(); + + ArrayType columnType = new ArrayType(VARCHAR); + PulsarColumnHandle columnHandle = new PulsarColumnHandle(getPulsarConnectorId().toString(), + "arrayField", columnType, false, false, "arrayField", + null, null, PulsarColumnHandle.HandleKeyValueType.NONE); + + checkArrayValues(getBlock(decodedRow, columnHandle), columnHandle.getType(), fieldValue); + } + + @Test + public void testMap() { + + DecoderTestMessage message = new DecoderTestMessage(); + message.mapField = new HashMap() {{ + put("key1", 2L); + put("key2", 22L); + }}; + + byte[] bytes = schema.encode(message); + ByteBuf payload = io.netty.buffer.Unpooled + .copiedBuffer(bytes); + GenericAvroRecord genericRecord = (GenericAvroRecord) GenericAvroSchema.of(schemaInfo).decode(bytes); + Object fieldValue = genericRecord.getAvroRecord().get("mapField"); + + Map decodedRow = pulsarRowDecoder.decodeRow(payload).get(); + Type columnType = decoderFactory.getTypeManager().getParameterizedType(StandardTypes.MAP, + ImmutableList.of(TypeSignatureParameter.typeParameter(VarcharType.VARCHAR.getTypeSignature()), + TypeSignatureParameter.typeParameter(BigintType.BIGINT.getTypeSignature()))); + + PulsarColumnHandle columnHandle = new PulsarColumnHandle(getPulsarConnectorId().toString(), "mapField", columnType, false, false, + "mapField", null, null, PulsarColumnHandle.HandleKeyValueType.NONE); + checkMapValues(getBlock(decodedRow, columnHandle), columnHandle.getType(), fieldValue); + } + + @Test + public void testCompositeType() { + DecoderTestMessage message = new DecoderTestMessage(); + + DecoderTestMessage.NestedRow nestedRow = new DecoderTestMessage.NestedRow(); + nestedRow.longField = 222L; + nestedRow.stringField = "message_2_nestedRow"; + + DecoderTestMessage.CompositeRow compositeRow = new DecoderTestMessage.CompositeRow(); + DecoderTestMessage.NestedRow nestedRow1 = new DecoderTestMessage.NestedRow(); + nestedRow1.longField = 2; + nestedRow1.stringField = "nestedRow_1"; + DecoderTestMessage.NestedRow nestedRow2 = new DecoderTestMessage.NestedRow(); + nestedRow2.longField = 2; + nestedRow2.stringField = "nestedRow_2"; + compositeRow.arrayField = Arrays.asList(nestedRow1, nestedRow2); + compositeRow.stringField = "compositeRow_1"; + + compositeRow.mapField = new HashMap() {{ + put("key1", nestedRow1); + put("key2", nestedRow2); + }}; + compositeRow.nestedRow = nestedRow; + + new HashMap() {{ + put("key1_1", 2L); + put("key1_2", 22L); + }}; + compositeRow.structedField = new HashMap>() {{ + put("key2_1", Arrays.asList(2L, 3L)); + put("key2_2", Arrays.asList(2L, 3L)); + put("key2_3", Arrays.asList(2L, 3L)); + }}; + + + message.compositeRow = compositeRow; + byte[] bytes = schema.encode(message); + ByteBuf payload = io.netty.buffer.Unpooled + .copiedBuffer(bytes); + GenericAvroRecord genericRecord = (GenericAvroRecord) GenericAvroSchema.of(schemaInfo).decode(bytes); + Object fieldValue = genericRecord.getAvroRecord().get("compositeRow"); + + Map decodedRow = pulsarRowDecoder.decodeRow(payload).get(); + + RowType columnType = RowType.from(ImmutableList.builder() + .add(RowType.field("stringField", VARCHAR)) + .add(RowType.field("arrayField", new ArrayType( + RowType.from(ImmutableList.builder() + .add(RowType.field("stringField", VARCHAR)) + .add(RowType.field("longField", BIGINT)) + .build())))) + .add(RowType.field("mapField", decoderFactory.getTypeManager().getParameterizedType(StandardTypes.MAP, + ImmutableList.of(TypeSignatureParameter.typeParameter(VarcharType.VARCHAR.getTypeSignature()), + TypeSignatureParameter.typeParameter(RowType.from(ImmutableList.builder() + .add(RowType.field("stringField", VARCHAR)) + .add(RowType.field("longField", BIGINT)) + .build()).getTypeSignature()) + )))) + .add(RowType.field("nestedRow", RowType.from(ImmutableList.builder() + .add(RowType.field("stringField", VARCHAR)) + .add(RowType.field("longField", BIGINT)) + .build()))) + .add(RowType.field("structedField", + decoderFactory.getTypeManager().getParameterizedType(StandardTypes.MAP, + ImmutableList.of(TypeSignatureParameter.typeParameter(VarcharType.VARCHAR.getTypeSignature()), + TypeSignatureParameter.typeParameter(new ArrayType(BIGINT).getTypeSignature()))))) + .build()); + + PulsarColumnHandle columnHandle = new PulsarColumnHandle(getPulsarConnectorId().toString(), + "compositeRow", columnType, false, false, "compositeRow", null, null, PulsarColumnHandle.HandleKeyValueType.NONE); + + checkRowValues(getBlock(decodedRow, columnHandle), columnHandle.getType(), fieldValue); + } + + @Test(singleThreaded = true) + public void testCyclicDefinitionDetect() { + AvroSchema cyclicSchema = AvroSchema.of(DecoderTestMessage.CyclicFoo.class); + PrestoException exception = expectThrows(PrestoException.class, + () -> { + decoderFactory.extractColumnMetadata(topicName, cyclicSchema.getSchemaInfo(), + PulsarColumnHandle.HandleKeyValueType.NONE); + }); + + assertEquals("Topic " + + topicName.toString() + " schema may contains cyclic definitions.", exception.getMessage()); + + } + +} diff --git a/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/decoder/json/JsonDecoderTestUtil.java b/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/decoder/json/JsonDecoderTestUtil.java new file mode 100644 index 0000000..f522a78 --- /dev/null +++ b/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/decoder/json/JsonDecoderTestUtil.java @@ -0,0 +1,199 @@ +/** + * 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.decoder.json; + +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.node.ArrayNode; +import com.fasterxml.jackson.databind.node.ObjectNode; +import com.google.common.collect.Iterators; +import io.prestosql.spi.block.Block; +import io.prestosql.spi.type.*; +import org.apache.pulsar.sql.presto.decoder.DecoderTestUtil; + +import java.io.IOException; +import java.util.Iterator; +import java.util.Map; + +import static io.prestosql.spi.type.VarcharType.VARCHAR; +import static java.lang.String.format; +import static org.testng.Assert.*; + +/** + * + * TestUtil for JsonDecoder + */ +public class JsonDecoderTestUtil extends DecoderTestUtil { + + public JsonDecoderTestUtil() { + super(); + } + + @Override + public void checkPrimitiveValue(Object actual, Object expected) { + assertTrue(expected instanceof JsonNode); + if (actual == null || null == expected) { + assertNull(expected); + assertNull(actual); + } else if (actual instanceof CharSequence) { + assertEquals(actual.toString(), ((JsonNode) expected).asText()); + } else if (actual instanceof SqlVarbinary) { + try { + assertEquals(((SqlVarbinary) actual).getBytes(), ((JsonNode) expected).binaryValue()); + } catch (IOException e) { + fail(format("JsonNode %s formate binary Value failed", ((JsonNode) expected).getNodeType().name())); + } + } else if (isIntegralType(actual)) { + assertEquals(((Number) actual).longValue(), ((JsonNode) expected).asLong()); + } else if (isRealType(actual)) { + assertEquals(((Number) actual).doubleValue(), ((JsonNode) expected).asDouble()); + } else { + assertEquals(actual, expected); + } + } + + @Override + public void checkMapValues(Block block, Type type, Object value) { + assertNotNull(type, "Type is null"); + assertTrue(type instanceof MapType, "Unexpected type"); + assertTrue(((MapType) type).getKeyType() instanceof VarcharType, "Unexpected key type"); + assertNotNull(block, "Block is null"); + assertNotNull(value, "Value is null"); + + assertTrue(value instanceof ObjectNode, "map node isn't ObjectNode type"); + + ObjectNode expected = (ObjectNode) value; + + Iterator> fields = expected.fields(); + + assertEquals(block.getPositionCount(), Iterators.size(expected.fields()) * 2); + Type valueType = ((MapType) type).getValueType(); + if (valueType instanceof ArrayType) { + for (int index = 0; index < block.getPositionCount(); index += 2) { + String actualKey = VARCHAR.getSlice(block, index).toStringUtf8(); + assertTrue(Iterators.any(fields, entry -> entry.getKey().equals(actualKey))); + if (block.isNull(index + 1)) { + assertNull(expected.get(actualKey)); + continue; + } + Block arrayBlock = block.getObject(index + 1, Block.class); + checkArrayValues(arrayBlock, valueType, expected.get(actualKey)); + } + } else if (valueType instanceof MapType) { + for (int index = 0; index < block.getPositionCount(); index += 2) { + String actualKey = VARCHAR.getSlice(block, index).toStringUtf8(); + assertTrue(Iterators.any(fields, entry -> entry.getKey().equals(actualKey))); + if (block.isNull(index + 1)) { + assertNull(expected.get(actualKey)); + continue; + } + Block mapBlock = block.getObject(index + 1, Block.class); + checkMapValues(mapBlock, valueType, expected.get(actualKey)); + } + } else if (valueType instanceof RowType) { + for (int index = 0; index < block.getPositionCount(); index += 2) { + String actualKey = VARCHAR.getSlice(block, index).toStringUtf8(); + assertTrue(Iterators.any(fields, entry -> entry.getKey().equals(actualKey))); + + if (block.isNull(index + 1)) { + assertNull(expected.get(actualKey)); + continue; + } + Block rowBlock = block.getObject(index + 1, Block.class); + checkRowValues(rowBlock, valueType, expected.get(actualKey)); + } + } else { + for (int index = 0; index < block.getPositionCount(); index += 2) { + String actualKey = VARCHAR.getSlice(block, index).toStringUtf8(); + Map.Entry entry = Iterators.tryFind(fields, e -> e.getKey().equals(actualKey)).get(); + assertNotNull(entry); + assertNotNull(entry.getKey()); + checkPrimitiveValue(getObjectValue(valueType, block, index + 1), entry.getValue()); + } + } + } + + @Override + public void checkRowValues(Block block, Type type, Object value) { + assertNotNull(type, "Type is null"); + assertTrue(type instanceof RowType, "Unexpected type"); + assertNotNull(block, "Block is null"); + assertNotNull(value, "Value is null"); + + ObjectNode record = (ObjectNode) value; + RowType rowType = (RowType) type; + assertEquals(Iterators.size(record.fields()), rowType.getFields().size(), "Json field size mismatch"); + assertEquals(block.getPositionCount(), rowType.getFields().size(), "Presto type field size mismatch"); + for (int fieldIndex = 0; fieldIndex < rowType.getFields().size(); fieldIndex++) { + RowType.Field rowField = rowType.getFields().get(fieldIndex); + Object expectedValue = record.get(rowField.getName().get()); + if (block.isNull(fieldIndex)) { + assertNull(expectedValue); + continue; + } + checkField(block, rowField.getType(), fieldIndex, expectedValue); + } + } + + @Override + public void checkArrayValues(Block block, Type type, Object value) { + assertNotNull(type, "Type is null"); + assertTrue(type instanceof ArrayType, "Unexpected type"); + assertNotNull(block, "Block is null"); + assertNotNull(value, "Value is null"); + + assertTrue(value instanceof ArrayNode, "Array node isn't ArrayNode type"); + ArrayNode arrayNode = (ArrayNode) value; + + assertEquals(block.getPositionCount(), arrayNode.size()); + Type elementType = ((ArrayType) type).getElementType(); + if (elementType instanceof ArrayType) { + for (int index = 0; index < block.getPositionCount(); index++) { + if (block.isNull(index)) { + assertNull(arrayNode.get(index)); + continue; + } + Block arrayBlock = block.getObject(index, Block.class); + checkArrayValues(arrayBlock, elementType, arrayNode.get(index)); + } + } else if (elementType instanceof MapType) { + for (int index = 0; index < block.getPositionCount(); index++) { + if (block.isNull(index)) { + assertNull(arrayNode.get(index)); + continue; + } + Block mapBlock = block.getObject(index, Block.class); + checkMapValues(mapBlock, elementType, arrayNode.get(index)); + } + } else if (elementType instanceof RowType) { + for (int index = 0; index < block.getPositionCount(); index++) { + if (block.isNull(index)) { + assertNull(arrayNode.get(index)); + continue; + } + Block rowBlock = block.getObject(index, Block.class); + checkRowValues(rowBlock, elementType, arrayNode.get(index)); + } + } else { + for (int index = 0; index < block.getPositionCount(); index++) { + checkPrimitiveValue(getObjectValue(elementType, block, index), arrayNode.get(index)); + } + } + } + +} diff --git a/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/decoder/json/TestJsonDecoder.java b/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/decoder/json/TestJsonDecoder.java new file mode 100644 index 0000000..9f56e30 --- /dev/null +++ b/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/decoder/json/TestJsonDecoder.java @@ -0,0 +1,284 @@ +/** + * 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.decoder.json; + +import com.google.common.collect.ImmutableList; +import io.netty.buffer.ByteBuf; +import io.prestosql.decoder.DecoderColumnHandle; +import io.prestosql.decoder.FieldValueProvider; +import io.prestosql.spi.PrestoException; +import io.prestosql.spi.type.*; +import org.apache.pulsar.client.impl.schema.JSONSchema; +import org.apache.pulsar.client.impl.schema.generic.GenericJsonRecord; +import org.apache.pulsar.client.impl.schema.generic.GenericJsonSchema; +import org.apache.pulsar.sql.presto.PulsarColumnHandle; +import org.apache.pulsar.sql.presto.decoder.AbstractDecoderTester; +import org.apache.pulsar.sql.presto.decoder.DecoderTestMessage; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Test; + +import java.time.LocalDate; +import java.time.LocalTime; +import java.time.ZoneId; +import java.time.temporal.ChronoUnit; +import java.util.*; + +import static io.prestosql.spi.type.BigintType.BIGINT; +import static io.prestosql.spi.type.BooleanType.BOOLEAN; +import static io.prestosql.spi.type.DoubleType.DOUBLE; +import static io.prestosql.spi.type.IntegerType.INTEGER; +import static io.prestosql.spi.type.RealType.REAL; +import static io.prestosql.spi.type.VarcharType.VARCHAR; +import static java.lang.Float.floatToIntBits; +import static org.apache.pulsar.sql.presto.TestPulsarConnector.getPulsarConnectorId; +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertTrue; +import static org.testng.Assert.expectThrows; + +public class TestJsonDecoder extends AbstractDecoderTester { + + private JSONSchema schema; + + @BeforeMethod + public void init() { + super.init(); + schema = JSONSchema.of(DecoderTestMessage.class); + schemaInfo = schema.getSchemaInfo(); + pulsarColumnHandle = getColumnColumnHandles(topicName, schemaInfo, PulsarColumnHandle.HandleKeyValueType.NONE, false, decoderFactory); + pulsarRowDecoder = decoderFactory.createRowDecoder(topicName, schemaInfo, new HashSet<>(pulsarColumnHandle)); + decoderTestUtil = new JsonDecoderTestUtil(); + assertTrue(pulsarRowDecoder instanceof PulsarJsonRowDecoder); + } + + @Test + public void testPrimitiveType() { + DecoderTestMessage message = new DecoderTestMessage(); + message.stringField = "message_1"; + message.intField = 22; + message.floatField = 2.2f; + message.doubleField = 22.20D; + message.booleanField = true; + message.longField = 222L; + message.timestampField = System.currentTimeMillis(); + message.enumField = DecoderTestMessage.TestEnum.TEST_ENUM_2; + + LocalTime now = LocalTime.now(ZoneId.systemDefault()); + message.timeField = now.toSecondOfDay() * 1000; + + LocalDate localDate = LocalDate.now(); + LocalDate epoch = LocalDate.ofEpochDay(0); + message.dateField = Math.toIntExact(ChronoUnit.DAYS.between(epoch, localDate)); + + ByteBuf payload = io.netty.buffer.Unpooled + .copiedBuffer(schema.encode(message)); + Map decodedRow = pulsarRowDecoder.decodeRow(payload).get(); + + PulsarColumnHandle stringFieldColumnHandle = new PulsarColumnHandle(getPulsarConnectorId().toString(), + "stringField", VARCHAR, false, false, "stringField", null, null, PulsarColumnHandle.HandleKeyValueType.NONE); + checkValue(decodedRow, stringFieldColumnHandle, message.stringField); + + PulsarColumnHandle intFieldColumnHandle = new PulsarColumnHandle(getPulsarConnectorId().toString(), + "intField", INTEGER, false, false, "intField", null, null, PulsarColumnHandle.HandleKeyValueType.NONE); + checkValue(decodedRow, intFieldColumnHandle, message.intField); + + PulsarColumnHandle floatFieldColumnHandle = new PulsarColumnHandle(getPulsarConnectorId().toString(), + "floatField", REAL, false, false, "floatField", null, null, PulsarColumnHandle.HandleKeyValueType.NONE); + checkValue(decodedRow, floatFieldColumnHandle, floatToIntBits(message.floatField)); + + PulsarColumnHandle doubleFieldColumnHandle = new PulsarColumnHandle(getPulsarConnectorId().toString(), + "doubleField", DOUBLE, false, false, "doubleField", null, null, PulsarColumnHandle.HandleKeyValueType.NONE); + checkValue(decodedRow, doubleFieldColumnHandle, message.doubleField); + + PulsarColumnHandle booleanFieldColumnHandle = new PulsarColumnHandle(getPulsarConnectorId().toString(), + "booleanField", BOOLEAN, false, false, "booleanField", null, null, PulsarColumnHandle.HandleKeyValueType.NONE); + checkValue(decodedRow, booleanFieldColumnHandle, message.booleanField); + + PulsarColumnHandle longFieldColumnHandle = new PulsarColumnHandle(getPulsarConnectorId().toString(), + "longField", BIGINT, false, false, "longField", null, null, PulsarColumnHandle.HandleKeyValueType.NONE); + checkValue(decodedRow, longFieldColumnHandle, message.longField); + + PulsarColumnHandle enumFieldColumnHandle = new PulsarColumnHandle(getPulsarConnectorId().toString(), + "enumField", VARCHAR, false, false, "enumField", null, null, PulsarColumnHandle.HandleKeyValueType.NONE); + checkValue(decodedRow, enumFieldColumnHandle, message.enumField.toString()); + + } + + @Test + public void testArray() { + DecoderTestMessage message = new DecoderTestMessage(); + message.arrayField = Arrays.asList("message_1", "message_2", "message_3"); + + byte[] bytes = schema.encode(message); + ByteBuf payload = io.netty.buffer.Unpooled + .copiedBuffer(bytes); + + GenericJsonRecord genericRecord = (GenericJsonRecord) GenericJsonSchema.of(schemaInfo).decode(bytes); + Object fieldValue = genericRecord.getJsonNode().get("arrayField"); + Map decodedRow = pulsarRowDecoder.decodeRow(payload).get(); + + ArrayType columnType = new ArrayType(VARCHAR); + PulsarColumnHandle columnHandle = new PulsarColumnHandle(getPulsarConnectorId().toString(), "arrayField", columnType, false, false, "arrayField", null, null, PulsarColumnHandle.HandleKeyValueType.NONE); + + checkArrayValues(getBlock(decodedRow, columnHandle), columnHandle.getType(), fieldValue); + } + + @Test + public void testRow() { + DecoderTestMessage message = new DecoderTestMessage(); + message.stringField = "message_2"; + DecoderTestMessage.TestRow testRow = new DecoderTestMessage.TestRow(); + message.rowField = testRow; + testRow.intField = 22; + testRow.stringField = "message_2_testRow"; + DecoderTestMessage.NestedRow nestedRow = new DecoderTestMessage.NestedRow(); + nestedRow.longField = 222L; + nestedRow.stringField = "message_2_nestedRow"; + testRow.nestedRow = nestedRow; + + byte[] bytes = schema.encode(message); + ByteBuf payload = io.netty.buffer.Unpooled + .copiedBuffer(bytes); + + GenericJsonRecord genericRecord = (GenericJsonRecord) GenericJsonSchema.of(schemaInfo).decode(bytes); + Object fieldValue = genericRecord.getJsonNode().get("rowField"); + Map decodedRow = pulsarRowDecoder.decodeRow(payload).get(); + + RowType columnType = RowType.from(ImmutableList.builder() + .add(RowType.field("stringField", VARCHAR)) + .add(RowType.field("intField", INTEGER)) + .add(RowType.field("nestedRow", RowType.from(ImmutableList.builder() + .add(RowType.field("stringField", VARCHAR)) + .add(RowType.field("longField", BIGINT)) + .build()))) + .build()); + + PulsarColumnHandle columnHandle = new PulsarColumnHandle(getPulsarConnectorId().toString(), "rowField", columnType, false, false, "rowField", null, null, PulsarColumnHandle.HandleKeyValueType.NONE); + + checkRowValues(getBlock(decodedRow, columnHandle), columnHandle.getType(), fieldValue); + + } + + @Test + public void testMap() { + DecoderTestMessage message = new DecoderTestMessage(); + message.mapField = new HashMap() {{ + put("key1", 2L); + put("key2", 22L); + }}; + + byte[] bytes = schema.encode(message); + ByteBuf payload = io.netty.buffer.Unpooled + .copiedBuffer(bytes); + + GenericJsonRecord genericRecord = (GenericJsonRecord) GenericJsonSchema.of(schemaInfo).decode(bytes); + Object fieldValue = genericRecord.getJsonNode().get("mapField"); + Map decodedRow = pulsarRowDecoder.decodeRow(payload).get(); + + Type columnType = decoderFactory.getTypeManager().getParameterizedType(StandardTypes.MAP, ImmutableList.of(TypeSignatureParameter.typeParameter(VarcharType.VARCHAR.getTypeSignature()), TypeSignatureParameter.typeParameter(BigintType.BIGINT.getTypeSignature()))); + PulsarColumnHandle columnHandle = new PulsarColumnHandle(getPulsarConnectorId().toString(), "mapField", columnType, false, false, "mapField", null, null, PulsarColumnHandle.HandleKeyValueType.NONE); + + checkMapValues(getBlock(decodedRow, columnHandle), columnHandle.getType(), fieldValue); + + } + + @Test + public void testCompositeType() { + DecoderTestMessage message = new DecoderTestMessage(); + + DecoderTestMessage.NestedRow nestedRow = new DecoderTestMessage.NestedRow(); + nestedRow.longField = 222L; + nestedRow.stringField = "message_2_nestedRow"; + + DecoderTestMessage.CompositeRow compositeRow = new DecoderTestMessage.CompositeRow(); + DecoderTestMessage.NestedRow nestedRow1 = new DecoderTestMessage.NestedRow(); + nestedRow1.longField = 2; + nestedRow1.stringField = "nestedRow_1"; + DecoderTestMessage.NestedRow nestedRow2 = new DecoderTestMessage.NestedRow(); + nestedRow2.longField = 2; + nestedRow2.stringField = "nestedRow_2"; + compositeRow.arrayField = Arrays.asList(nestedRow1, nestedRow2); + compositeRow.stringField = "compositeRow_1"; + + compositeRow.mapField = new HashMap() {{ + put("key1", nestedRow1); + put("key2", nestedRow2); + }}; + compositeRow.nestedRow = nestedRow; + new HashMap() {{ + put("key1_1", 2L); + put("key1_2", 22L); + }}; + compositeRow.structedField = new HashMap>() {{ + put("key2_1", Arrays.asList(2L, 3L)); + put("key2_2", Arrays.asList(2L, 3L)); + put("key2_3", Arrays.asList(2L, 3L)); + }}; + message.compositeRow = compositeRow; + + byte[] bytes = schema.encode(message); + ByteBuf payload = io.netty.buffer.Unpooled + .copiedBuffer(bytes); + GenericJsonRecord genericRecord = (GenericJsonRecord) GenericJsonSchema.of(schemaInfo).decode(bytes); + Object fieldValue = genericRecord.getJsonNode().get("compositeRow"); + Map decodedRow = pulsarRowDecoder.decodeRow(payload).get(); + + RowType columnType = RowType.from(ImmutableList.builder() + .add(RowType.field("stringField", VARCHAR)) + .add(RowType.field("arrayField", new ArrayType( + RowType.from(ImmutableList.builder() + .add(RowType.field("stringField", VARCHAR)) + .add(RowType.field("longField", BIGINT)) + .build())))) + .add(RowType.field("mapField", decoderFactory.getTypeManager().getParameterizedType(StandardTypes.MAP, + ImmutableList.of(TypeSignatureParameter.typeParameter(VarcharType.VARCHAR.getTypeSignature()), + TypeSignatureParameter.typeParameter(RowType.from(ImmutableList.builder() + .add(RowType.field("stringField", VARCHAR)) + .add(RowType.field("longField", BIGINT)) + .build()).getTypeSignature()) + )))) + .add(RowType.field("nestedRow", RowType.from(ImmutableList.builder() + .add(RowType.field("stringField", VARCHAR)) + .add(RowType.field("longField", BIGINT)) + .build()))) + .add(RowType.field("structedField", + decoderFactory.getTypeManager().getParameterizedType(StandardTypes.MAP, + ImmutableList.of(TypeSignatureParameter.typeParameter(VarcharType.VARCHAR.getTypeSignature()), + TypeSignatureParameter.typeParameter(new ArrayType(BIGINT).getTypeSignature()))))) + .build()); + + PulsarColumnHandle columnHandle = new PulsarColumnHandle(getPulsarConnectorId().toString(), "compositeRow", columnType, false, false, "compositeRow", null, null, PulsarColumnHandle.HandleKeyValueType.NONE); + + checkRowValues(getBlock(decodedRow, columnHandle), columnHandle.getType(), fieldValue); + } + + @Test(singleThreaded = true) + public void testCyclicDefinitionDetect() { + JSONSchema cyclicSchema = JSONSchema.of(DecoderTestMessage.CyclicFoo.class); + PrestoException exception = expectThrows(PrestoException.class, + () -> { + decoderFactory.extractColumnMetadata(topicName, cyclicSchema.getSchemaInfo(), + PulsarColumnHandle.HandleKeyValueType.NONE); + }); + + assertEquals("Topic " + + topicName.toString() + " schema may contains cyclic definitions.", exception.getMessage()); + + } + +} diff --git a/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/decoder/primitive/PrimitiveDecoderTestUtil.java b/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/decoder/primitive/PrimitiveDecoderTestUtil.java new file mode 100644 index 0000000..53813d5 --- /dev/null +++ b/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/decoder/primitive/PrimitiveDecoderTestUtil.java @@ -0,0 +1,56 @@ +/** + * 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.decoder.primitive; + +import io.prestosql.spi.block.Block; +import io.prestosql.spi.type.Type; +import org.apache.pulsar.sql.presto.decoder.DecoderTestUtil; + +/** + * TestUtil for PrimitiveDecoder. + * CheckXXXValues() is mock method. Because Primitive is single hierarchy, so CheckXXXValues are never actually + * invoked. + */ +public class PrimitiveDecoderTestUtil extends DecoderTestUtil { + + public PrimitiveDecoderTestUtil() { + super(); + } + + @Override + public void checkArrayValues(Block block, Type type, Object value) { + + } + + @Override + public void checkMapValues(Block block, Type type, Object value) { + + } + + @Override + public void checkRowValues(Block block, Type type, Object value) { + + } + + @Override + public void checkPrimitiveValue(Object actual, Object expected) { + + } + +} diff --git a/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/decoder/primitive/TestPrimitiveDecoder.java b/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/decoder/primitive/TestPrimitiveDecoder.java new file mode 100644 index 0000000..f4810ba --- /dev/null +++ b/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/decoder/primitive/TestPrimitiveDecoder.java @@ -0,0 +1,233 @@ +/** + * 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.decoder.primitive; + +import io.airlift.slice.Slices; +import io.prestosql.decoder.DecoderColumnHandle; +import io.prestosql.decoder.FieldValueProvider; +import org.apache.pulsar.client.api.Schema; +import org.apache.pulsar.common.schema.SchemaInfo; +import org.apache.pulsar.common.schema.SchemaType; +import org.apache.pulsar.sql.presto.PulsarColumnHandle; +import org.apache.pulsar.sql.presto.PulsarRowDecoder; +import org.apache.pulsar.sql.presto.decoder.AbstractDecoderTester; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Test; + +import java.sql.Time; +import java.sql.Timestamp; +import java.util.Date; +import java.util.HashSet; +import java.util.List; +import java.util.Map; + +import static io.prestosql.spi.type.DateType.DATE; +import static io.prestosql.spi.type.SmallintType.SMALLINT; +import static io.prestosql.spi.type.TimeType.TIME; +import static io.prestosql.spi.type.TimestampType.TIMESTAMP; +import static io.prestosql.spi.type.TinyintType.TINYINT; +import static io.prestosql.spi.type.VarbinaryType.VARBINARY; +import static io.prestosql.spi.type.VarcharType.VARCHAR; +import static io.prestosql.spi.type.BigintType.BIGINT; +import static io.prestosql.spi.type.BooleanType.BOOLEAN; +import static io.prestosql.spi.type.DoubleType.DOUBLE; +import static io.prestosql.spi.type.IntegerType.INTEGER; +import static io.prestosql.spi.type.RealType.REAL; +import static org.apache.pulsar.sql.presto.TestPulsarConnector.getPulsarConnectorId; + +public class TestPrimitiveDecoder extends AbstractDecoderTester { + + public static final String PRIMITIVE_COLUMN_NAME = "__value__"; + + @BeforeMethod + public void init() { + decoderTestUtil = new PrimitiveDecoderTestUtil(); + super.init(); + } + + @Test(singleThreaded = true) + public void testPrimitiveType() { + + byte int8Value = 1; + SchemaInfo schemaInfoInt8 = SchemaInfo.builder().type(SchemaType.INT8).build(); + Schema schemaInt8 = Schema.getSchema(schemaInfoInt8); + List pulsarColumnHandleInt8 = getColumnColumnHandles(topicName, schemaInfoInt8, PulsarColumnHandle.HandleKeyValueType.NONE, false, decoderFactory); + PulsarRowDecoder pulsarRowDecoderInt8 = decoderFactory.createRowDecoder(topicName, schemaInfoInt8, + new HashSet<>(pulsarColumnHandleInt8)); + Map decodedRowInt8 = + pulsarRowDecoderInt8.decodeRow(io.netty.buffer.Unpooled + .copiedBuffer(schemaInt8.encode(int8Value))).get(); + checkValue(decodedRowInt8, new PulsarColumnHandle(getPulsarConnectorId().toString(), + PRIMITIVE_COLUMN_NAME, TINYINT, false, false, PRIMITIVE_COLUMN_NAME, null, null, PulsarColumnHandle.HandleKeyValueType.NONE), int8Value); + + short int16Value = 2; + SchemaInfo schemaInfoInt16 = SchemaInfo.builder().type(SchemaType.INT16).build(); + Schema schemaInt16 = Schema.getSchema(schemaInfoInt16); + List pulsarColumnHandleInt16 = getColumnColumnHandles(topicName, schemaInfoInt16, PulsarColumnHandle.HandleKeyValueType.NONE, false, decoderFactory); + PulsarRowDecoder pulsarRowDecoderInt16 = decoderFactory.createRowDecoder(topicName, schemaInfoInt16, + new HashSet<>(pulsarColumnHandleInt16)); + Map decodedRowInt16 = + pulsarRowDecoderInt16.decodeRow(io.netty.buffer.Unpooled + .copiedBuffer(schemaInt16.encode(int16Value))).get(); + checkValue(decodedRowInt16, new PulsarColumnHandle(getPulsarConnectorId().toString(), + PRIMITIVE_COLUMN_NAME, SMALLINT, false, false, PRIMITIVE_COLUMN_NAME, null, null, PulsarColumnHandle.HandleKeyValueType.NONE), int16Value); + + int int32Value = 2; + SchemaInfo schemaInfoInt32 = SchemaInfo.builder().type(SchemaType.INT32).build(); + Schema schemaInt32 = Schema.getSchema(schemaInfoInt32); + List pulsarColumnHandleInt32 = getColumnColumnHandles(topicName, schemaInfoInt32, + PulsarColumnHandle.HandleKeyValueType.NONE, false, decoderFactory); + PulsarRowDecoder pulsarRowDecoderInt32 = decoderFactory.createRowDecoder(topicName, schemaInfoInt32, + new HashSet<>(pulsarColumnHandleInt32)); + Map decodedRowInt32 = + pulsarRowDecoderInt32.decodeRow(io.netty.buffer.Unpooled + .copiedBuffer(schemaInt32.encode(int32Value))).get(); + checkValue(decodedRowInt32, new PulsarColumnHandle(getPulsarConnectorId().toString(), + PRIMITIVE_COLUMN_NAME, INTEGER, false, false, PRIMITIVE_COLUMN_NAME, null, null, PulsarColumnHandle.HandleKeyValueType.NONE), int32Value); + + long int64Value = 2; + SchemaInfo schemaInfoInt64 = SchemaInfo.builder().type(SchemaType.INT64).build(); + Schema schemaInt64 = Schema.getSchema(schemaInfoInt64); + List pulsarColumnHandleInt64 = getColumnColumnHandles(topicName, schemaInfoInt64, + PulsarColumnHandle.HandleKeyValueType.NONE, false, decoderFactory); + PulsarRowDecoder pulsarRowDecoderInt64 = decoderFactory.createRowDecoder(topicName, schemaInfoInt64, + new HashSet<>(pulsarColumnHandleInt64)); + Map decodedRowInt64 = + pulsarRowDecoderInt64.decodeRow(io.netty.buffer.Unpooled + .copiedBuffer(schemaInt64.encode(int64Value))).get(); + checkValue(decodedRowInt64, new PulsarColumnHandle(getPulsarConnectorId().toString(), + PRIMITIVE_COLUMN_NAME, BIGINT, false, false, PRIMITIVE_COLUMN_NAME, null, null, + PulsarColumnHandle.HandleKeyValueType.NONE), int64Value); + + String stringValue = "test"; + SchemaInfo schemaInfoString = SchemaInfo.builder().type(SchemaType.STRING).build(); + Schema schemaString = Schema.getSchema(schemaInfoString); + List pulsarColumnHandleString = getColumnColumnHandles(topicName, schemaInfoString, + PulsarColumnHandle.HandleKeyValueType.NONE, false, decoderFactory); + PulsarRowDecoder pulsarRowDecoderString = decoderFactory.createRowDecoder(topicName, schemaInfoString, + new HashSet<>(pulsarColumnHandleString)); + Map decodedRowString = + pulsarRowDecoderString.decodeRow(io.netty.buffer.Unpooled + .copiedBuffer(schemaString.encode(stringValue))).get(); + checkValue(decodedRowString, new PulsarColumnHandle(getPulsarConnectorId().toString(), + PRIMITIVE_COLUMN_NAME, VARCHAR, false, false, PRIMITIVE_COLUMN_NAME, null, null, + PulsarColumnHandle.HandleKeyValueType.NONE), stringValue); + + float floatValue = 0.2f; + SchemaInfo schemaInfoFloat = SchemaInfo.builder().type(SchemaType.FLOAT).build(); + Schema schemaFloat = Schema.getSchema(schemaInfoFloat); + List pulsarColumnHandleFloat = getColumnColumnHandles(topicName, schemaInfoFloat, + PulsarColumnHandle.HandleKeyValueType.NONE, false, decoderFactory); + PulsarRowDecoder pulsarRowDecoderFloat = decoderFactory.createRowDecoder(topicName, schemaInfoFloat, + new HashSet<>(pulsarColumnHandleFloat)); + Map decodedRowFloat = + pulsarRowDecoderFloat.decodeRow(io.netty.buffer.Unpooled + .copiedBuffer(schemaFloat.encode(floatValue))).get(); + checkValue(decodedRowFloat, new PulsarColumnHandle(getPulsarConnectorId().toString(), + PRIMITIVE_COLUMN_NAME, REAL, false, false, PRIMITIVE_COLUMN_NAME, null, null, + PulsarColumnHandle.HandleKeyValueType.NONE), Long.valueOf(Float.floatToIntBits(floatValue))); + + double doubleValue = 0.22d; + SchemaInfo schemaInfoDouble = SchemaInfo.builder().type(SchemaType.DOUBLE).build(); + Schema schemaDouble = Schema.getSchema(schemaInfoDouble); + List pulsarColumnHandleDouble = getColumnColumnHandles(topicName, schemaInfoDouble, + PulsarColumnHandle.HandleKeyValueType.NONE, false, decoderFactory); + PulsarRowDecoder pulsarRowDecoderDouble = decoderFactory.createRowDecoder(topicName, schemaInfoDouble, + new HashSet<>(pulsarColumnHandleDouble)); + Map decodedRowDouble = + pulsarRowDecoderDouble.decodeRow(io.netty.buffer.Unpooled + .copiedBuffer(schemaDouble.encode(doubleValue))).get(); + checkValue(decodedRowDouble, new PulsarColumnHandle(getPulsarConnectorId().toString(), + PRIMITIVE_COLUMN_NAME, DOUBLE, false, false, PRIMITIVE_COLUMN_NAME, null, null, + PulsarColumnHandle.HandleKeyValueType.NONE), doubleValue); + + boolean booleanValue = true; + SchemaInfo schemaInfoBoolean = SchemaInfo.builder().type(SchemaType.BOOLEAN).build(); + Schema schemaBoolean = Schema.getSchema(schemaInfoBoolean); + List pulsarColumnHandleBoolean = getColumnColumnHandles(topicName, schemaInfoBoolean, + PulsarColumnHandle.HandleKeyValueType.NONE, false, decoderFactory); + PulsarRowDecoder pulsarRowDecoderBoolean = decoderFactory.createRowDecoder(topicName, schemaInfoBoolean, + new HashSet<>(pulsarColumnHandleBoolean)); + Map decodedRowBoolean = + pulsarRowDecoderBoolean.decodeRow(io.netty.buffer.Unpooled + .copiedBuffer(schemaBoolean.encode(booleanValue))).get(); + checkValue(decodedRowBoolean, new PulsarColumnHandle(getPulsarConnectorId().toString(), + PRIMITIVE_COLUMN_NAME, BOOLEAN, false, false, PRIMITIVE_COLUMN_NAME, null, null, + PulsarColumnHandle.HandleKeyValueType.NONE), booleanValue); + + byte[] bytesValue = new byte[1]; + bytesValue[0] = 1; + SchemaInfo schemaInfoBytes = SchemaInfo.builder().type(SchemaType.BYTES).build(); + Schema schemaBytes = Schema.getSchema(schemaInfoBytes); + List pulsarColumnHandleBytes = getColumnColumnHandles(topicName, schemaInfoBytes, + PulsarColumnHandle.HandleKeyValueType.NONE, false, decoderFactory); + PulsarRowDecoder pulsarRowDecoderBytes = decoderFactory.createRowDecoder(topicName, schemaInfoBytes, + new HashSet<>(pulsarColumnHandleBytes)); + Map decodedRowBytes = + pulsarRowDecoderBytes.decodeRow(io.netty.buffer.Unpooled + .copiedBuffer(schemaBytes.encode(bytesValue))).get(); + checkValue(decodedRowBytes, new PulsarColumnHandle(getPulsarConnectorId().toString(), + PRIMITIVE_COLUMN_NAME, VARBINARY, false, false, PRIMITIVE_COLUMN_NAME, null, null, + PulsarColumnHandle.HandleKeyValueType.NONE), Slices.wrappedBuffer(bytesValue)); + + Date dateValue = new Date(System.currentTimeMillis()); + SchemaInfo schemaInfoDate = SchemaInfo.builder().type(SchemaType.DATE).build(); + Schema schemaDate = Schema.getSchema(schemaInfoDate); + List pulsarColumnHandleDate = getColumnColumnHandles(topicName, schemaInfoDate, + PulsarColumnHandle.HandleKeyValueType.NONE, false, decoderFactory); + PulsarRowDecoder pulsarRowDecoderDate = decoderFactory.createRowDecoder(topicName, schemaInfoDate, + new HashSet<>(pulsarColumnHandleDate)); + Map decodedRowDate = + pulsarRowDecoderDate.decodeRow(io.netty.buffer.Unpooled + .copiedBuffer(schemaDate.encode(dateValue))).get(); + checkValue(decodedRowDate, new PulsarColumnHandle(getPulsarConnectorId().toString(), + PRIMITIVE_COLUMN_NAME, DATE, false, false, PRIMITIVE_COLUMN_NAME, null, null, + PulsarColumnHandle.HandleKeyValueType.NONE), dateValue.getTime()); + + Time timeValue = new Time(System.currentTimeMillis()); + SchemaInfo schemaInfoTime = SchemaInfo.builder().type(SchemaType.TIME).build(); + Schema schemaTime = Schema.getSchema(schemaInfoTime); + List pulsarColumnHandleTime = getColumnColumnHandles(topicName, schemaInfoTime, + PulsarColumnHandle.HandleKeyValueType.NONE, false, decoderFactory); + PulsarRowDecoder pulsarRowDecoderTime = decoderFactory.createRowDecoder(topicName, schemaInfoTime, + new HashSet<>(pulsarColumnHandleTime)); + Map decodedRowTime = + pulsarRowDecoderTime.decodeRow(io.netty.buffer.Unpooled + .copiedBuffer(schemaTime.encode(timeValue))).get(); + checkValue(decodedRowTime, new PulsarColumnHandle(getPulsarConnectorId().toString(), + PRIMITIVE_COLUMN_NAME, TIME, false, false, PRIMITIVE_COLUMN_NAME, null, null, + PulsarColumnHandle.HandleKeyValueType.NONE), timeValue.getTime()); + + Timestamp timestampValue = new Timestamp(System.currentTimeMillis()); + SchemaInfo schemaInfoTimestamp = SchemaInfo.builder().type(SchemaType.TIMESTAMP).build(); + Schema schemaTimestamp = Schema.getSchema(schemaInfoTimestamp); + List pulsarColumnHandleTimestamp = getColumnColumnHandles(topicName, schemaInfoTimestamp, + PulsarColumnHandle.HandleKeyValueType.NONE, false, decoderFactory); + PulsarRowDecoder pulsarRowDecoderTimestamp = decoderFactory.createRowDecoder(topicName, schemaInfoTimestamp, + new HashSet<>(pulsarColumnHandleTimestamp)); + Map decodedRowTimestamp = + pulsarRowDecoderTimestamp.decodeRow(io.netty.buffer.Unpooled + .copiedBuffer(schemaTimestamp.encode(timestampValue))).get(); + checkValue(decodedRowTimestamp, new PulsarColumnHandle(getPulsarConnectorId().toString(), + PRIMITIVE_COLUMN_NAME, TIMESTAMP, false, false, PRIMITIVE_COLUMN_NAME, null, null, + PulsarColumnHandle.HandleKeyValueType.NONE), timestampValue.getTime()); + + } + +}