gaoran10 commented on a change in pull request #9841: URL: https://github.com/apache/pulsar/pull/9841#discussion_r591204342
########## File path: pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/decoder/protobufnative/PulsarProtobufNativeRowDecoder.java ########## @@ -0,0 +1,76 @@ +/** + * 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.protobufnative; + +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 com.google.protobuf.DynamicMessage; +import io.netty.buffer.ByteBuf; +import io.prestosql.decoder.DecoderColumnHandle; +import io.prestosql.decoder.FieldValueProvider; +import io.prestosql.spi.PrestoException; +import org.apache.pulsar.client.impl.schema.generic.GenericProtobufNativeRecord; +import org.apache.pulsar.client.impl.schema.generic.GenericProtobufNativeSchema; +import org.apache.pulsar.sql.presto.PulsarRowDecoder; +import java.util.Map; +import java.util.Optional; +import java.util.Set; + +/** + * Pulsar {@link org.apache.pulsar.common.schema.SchemaType#PROTOBUF_NATIVE} RowDecoder. + */ +public class PulsarProtobufNativeRowDecoder implements PulsarRowDecoder { + + private final GenericProtobufNativeSchema genericProtobufNativeSchema; + private final Map<DecoderColumnHandle, PulsarProtobufNativeColumnDecoder> columnDecoders; + + public PulsarProtobufNativeRowDecoder(GenericProtobufNativeSchema genericProtobufNativeSchema, Set<DecoderColumnHandle> columns) { + this.genericProtobufNativeSchema = requireNonNull(genericProtobufNativeSchema, "genericProtobufNativeSchema is null"); + columnDecoders = columns.stream() + .collect(toImmutableMap(identity(), this::createColumnDecoder)); + } + + private PulsarProtobufNativeColumnDecoder createColumnDecoder(DecoderColumnHandle columnHandle) { + return new PulsarProtobufNativeColumnDecoder(columnHandle); + } + + /** + * Decode ByteBuf by {@link org.apache.pulsar.client.api.schema.GenericSchema}. + * @param byteBuf + * @return + */ + @Override + public Optional<Map<DecoderColumnHandle, FieldValueProvider>> decodeRow(ByteBuf byteBuf) { + DynamicMessage dynamicMessage; + try { + GenericProtobufNativeRecord record = (GenericProtobufNativeRecord) genericProtobufNativeSchema.decode(byteBuf); + dynamicMessage = record.getProtobufRecord(); + } catch (Exception e) { + e.printStackTrace(); Review comment: Could you use the method `log.error()` instead of the `e.printStackTrace()`? ########## File path: pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/decoder/protobufnative/PulsarProtobufNativeColumnDecoder.java ########## @@ -0,0 +1,386 @@ +/** + * 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.protobufnative; + +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 com.google.protobuf.ByteString; +import com.google.protobuf.DynamicMessage; +import com.google.protobuf.EnumValue; +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.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.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.List; +import java.util.Map; +import java.util.Set; + +/** + * Pulsar {@link org.apache.pulsar.common.schema.SchemaType#PROTOBUF_NATIVE} ColumnDecoder. + */ +public class PulsarProtobufNativeColumnDecoder { + private static final Set<Type> SUPPORTED_PRIMITIVE_TYPES = ImmutableSet.of( + BooleanType.BOOLEAN, + IntegerType.INTEGER, + BigintType.BIGINT, + RealType.REAL, + DoubleType.DOUBLE, + VarbinaryType.VARBINARY); + + private final Type columnType; + private final String columnMapping; + private final String columnName; + + public PulsarProtobufNativeColumnDecoder(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<Type> typeParameters = type.getTypeParameters(); + checkArgument(typeParameters.size() == 2, + "expecting exactly two type parameters for map"); + return isSupportedType(type.getTypeParameters().get(1)) && isSupportedType(type.getTypeParameters().get(0)); + } + + 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); Review comment: Could we add the `VarcharType` to the set `SUPPORTED_PRIMITIVE_TYPES`? ########## File path: pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/decoder/protobufnative/PulsarProtobufNativeColumnDecoder.java ########## @@ -0,0 +1,386 @@ +/** + * 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.protobufnative; + +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 com.google.protobuf.ByteString; +import com.google.protobuf.DynamicMessage; +import com.google.protobuf.EnumValue; +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.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.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.List; +import java.util.Map; +import java.util.Set; + +/** + * Pulsar {@link org.apache.pulsar.common.schema.SchemaType#PROTOBUF_NATIVE} ColumnDecoder. + */ +public class PulsarProtobufNativeColumnDecoder { + private static final Set<Type> SUPPORTED_PRIMITIVE_TYPES = ImmutableSet.of( + BooleanType.BOOLEAN, + IntegerType.INTEGER, + BigintType.BIGINT, + RealType.REAL, + DoubleType.DOUBLE, + VarbinaryType.VARBINARY); + + private final Type columnType; + private final String columnMapping; + private final String columnName; + + public PulsarProtobufNativeColumnDecoder(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<Type> typeParameters = type.getTypeParameters(); + checkArgument(typeParameters.size() == 2, + "expecting exactly two type parameters for map"); + return isSupportedType(type.getTypeParameters().get(1)) && isSupportedType(type.getTypeParameters().get(0)); Review comment: A trivial point, maybe we could use the the `typeParameters.get(1) instead of the `type.getTypeParameters().get(1)`. ---------------------------------------------------------------- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: [email protected]
