piotr-szuberski commented on a change in pull request #13572: URL: https://github.com/apache/beam/pull/13572#discussion_r546247664
########## File path: sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/kafka/BeamKafkaThriftTable.java ########## @@ -0,0 +1,128 @@ +/* + * 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.beam.sdk.extensions.sql.meta.provider.kafka; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.util.List; +import org.apache.beam.sdk.io.thrift.ThriftCoder; +import org.apache.beam.sdk.io.thrift.ThriftSchema; +import org.apache.beam.sdk.schemas.Schema; +import org.apache.beam.sdk.schemas.SchemaProvider; +import org.apache.beam.sdk.schemas.transforms.Convert; +import org.apache.beam.sdk.transforms.MapElements; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.transforms.Values; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.Row; +import org.apache.beam.sdk.values.TypeDescriptor; +import org.apache.beam.sdk.values.TypeDescriptors; +import org.apache.thrift.TBase; +import org.apache.thrift.TFieldIdEnum; +import org.apache.thrift.protocol.TProtocolFactory; +import org.checkerframework.checker.nullness.qual.NonNull; + +public class BeamKafkaThriftTable<FieldT extends TFieldIdEnum, T extends TBase<T, FieldT>> + extends BeamKafkaTable { + private final ThriftCoder<T> thriftCoder; + private final TypeDescriptor<T> typeDescriptor; + + public BeamKafkaThriftTable( + @NonNull Schema requiredSchema, + @NonNull String bootstrapServers, + @NonNull List<String> topics, + @NonNull Class<T> thriftClass, + @NonNull TProtocolFactory protocolFactory) { + super(thriftSchema(thriftClass, requiredSchema), bootstrapServers, topics); + typeDescriptor = TypeDescriptor.of(thriftClass); + thriftCoder = ThriftCoder.of(thriftClass, protocolFactory); + } + + private static Schema thriftSchema( + @NonNull Class<?> thriftClass, @NonNull Schema requiredSchema) { + @SuppressWarnings("nullness") + final @NonNull Schema thriftSchema = + ThriftSchema.provider().schemaFor(TypeDescriptor.of(thriftClass)); + if (!requiredSchema.equivalent(thriftSchema)) { + throw new IllegalArgumentException( + String.format( + "Given message schema: '%s'%n" + + "does not match schema inferred from thrift class.%n" + + "Thrift class: '%s'%n" + + "Inferred schema: '%s'", + requiredSchema, thriftClass.getName(), thriftSchema)); + } + return thriftSchema; + } + + @Override + protected PTransform<PCollection<KV<byte[], byte[]>>, PCollection<Row>> getPTransformForInput() { + final @NonNull SchemaProvider schemaProvider = ThriftSchema.provider(); + return new PTransform<PCollection<KV<byte[], byte[]>>, PCollection<Row>>() { + @Override + @SuppressWarnings("nullness") + public PCollection<Row> expand(PCollection<KV<byte[], byte[]>> input) { + return input + .apply(Values.create()) + .apply(MapElements.into(typeDescriptor).via(BeamKafkaThriftTable.this::decode)) Review comment: Just one more comment. It would be good to have some function or DoFndeclared like it is done for protobuf so we don't have to repeat methods like encode and decode together with ThriftCoder in every table provider that supports thrift. We could then just import a function returning SimpleFunction/DoFnand use it like `MapElements.via(thriftBytesToBeamRow(thriftClass, protocolFactory))` or `ParDo.of(thriftBytesToBeamRowFn)` ########## File path: sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/kafka/BeamKafkaThriftTable.java ########## @@ -0,0 +1,128 @@ +/* + * 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.beam.sdk.extensions.sql.meta.provider.kafka; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.util.List; +import org.apache.beam.sdk.io.thrift.ThriftCoder; +import org.apache.beam.sdk.io.thrift.ThriftSchema; +import org.apache.beam.sdk.schemas.Schema; +import org.apache.beam.sdk.schemas.SchemaProvider; +import org.apache.beam.sdk.schemas.transforms.Convert; +import org.apache.beam.sdk.transforms.MapElements; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.transforms.Values; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.Row; +import org.apache.beam.sdk.values.TypeDescriptor; +import org.apache.beam.sdk.values.TypeDescriptors; +import org.apache.thrift.TBase; +import org.apache.thrift.TFieldIdEnum; +import org.apache.thrift.protocol.TProtocolFactory; +import org.checkerframework.checker.nullness.qual.NonNull; + +public class BeamKafkaThriftTable<FieldT extends TFieldIdEnum, T extends TBase<T, FieldT>> + extends BeamKafkaTable { + private final ThriftCoder<T> thriftCoder; + private final TypeDescriptor<T> typeDescriptor; + + public BeamKafkaThriftTable( + @NonNull Schema requiredSchema, + @NonNull String bootstrapServers, + @NonNull List<String> topics, + @NonNull Class<T> thriftClass, + @NonNull TProtocolFactory protocolFactory) { + super(thriftSchema(thriftClass, requiredSchema), bootstrapServers, topics); + typeDescriptor = TypeDescriptor.of(thriftClass); + thriftCoder = ThriftCoder.of(thriftClass, protocolFactory); + } + + private static Schema thriftSchema( + @NonNull Class<?> thriftClass, @NonNull Schema requiredSchema) { + @SuppressWarnings("nullness") + final @NonNull Schema thriftSchema = + ThriftSchema.provider().schemaFor(TypeDescriptor.of(thriftClass)); + if (!requiredSchema.equivalent(thriftSchema)) { + throw new IllegalArgumentException( + String.format( + "Given message schema: '%s'%n" + + "does not match schema inferred from thrift class.%n" + + "Thrift class: '%s'%n" + + "Inferred schema: '%s'", + requiredSchema, thriftClass.getName(), thriftSchema)); + } + return thriftSchema; + } + + @Override + protected PTransform<PCollection<KV<byte[], byte[]>>, PCollection<Row>> getPTransformForInput() { + final @NonNull SchemaProvider schemaProvider = ThriftSchema.provider(); + return new PTransform<PCollection<KV<byte[], byte[]>>, PCollection<Row>>() { + @Override + @SuppressWarnings("nullness") + public PCollection<Row> expand(PCollection<KV<byte[], byte[]>> input) { + return input + .apply(Values.create()) + .apply(MapElements.into(typeDescriptor).via(BeamKafkaThriftTable.this::decode)) Review comment: Just one more comment. It would be good to have some function or DoFndeclared like it is done for protobuf so we don't have to repeat methods like encode and decode together with ThriftCoder in every table provider that supports thrift. We could then just import a function returning SimpleFunction/DoFnand use it like `MapElements.via(thriftBytesToBeamRowFn(thriftClass, protocolFactory))` or `ParDo.of(thriftBytesToBeamRowFn)` ---------------------------------------------------------------- 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: us...@infra.apache.org