[ 
https://issues.apache.org/jira/browse/BEAM-11482?focusedWorklogId=526293&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-526293
 ]

ASF GitHub Bot logged work on BEAM-11482:
-----------------------------------------

                Author: ASF GitHub Bot
            Created on: 19/Dec/20 15:06
            Start Date: 19/Dec/20 15:06
    Worklog Time Spent: 10m 
      Work Description: 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(thriftBytesToBeamRowFn(thriftClass, protocolFactory))` or 
`ParDo.of(thriftBytesToBeamRowFn)`
   
   Protobuf example: 
https://github.com/apache/beam/blob/1466db958ca22222ae8badf1f21a860267d46d31/sdks/java/extensions/protobuf/src/main/java/org/apache/beam/sdk/extensions/protobuf/ProtoMessageSchema.java#L137




----------------------------------------------------------------
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


Issue Time Tracking
-------------------

    Worklog Id:     (was: 526293)
    Time Spent: 1h 20m  (was: 1h 10m)

> Thrift support for KafkaTableProvider
> -------------------------------------
>
>                 Key: BEAM-11482
>                 URL: https://issues.apache.org/jira/browse/BEAM-11482
>             Project: Beam
>          Issue Type: New Feature
>          Components: dsl-sql, io-java-kafka
>            Reporter: Costi Ciudatu
>            Assignee: Costi Ciudatu
>            Priority: P2
>          Time Spent: 1h 20m
>  Remaining Estimate: 0h
>
> Kafka table provider can leverage the Thrift coder and schema provider 
> defined in the IO package to handle thrift input/output.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to