hnail commented on a change in pull request #8422:
URL: https://github.com/apache/pulsar/pull/8422#discussion_r547783095



##########
File path: 
pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/decoder/avro/PulsarAvroRowDecoderFactory.java
##########
@@ -0,0 +1,179 @@
+/**
+ * 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.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<DecoderColumnHandle> columns) 
{
+        return new PulsarAvroRowDecoder((GenericAvroSchema) 
GenericAvroSchema.of(schemaInfo), columns);
+    }
+
+    @Override
+    public List<ColumnMetadata> extractColumnMetadata(TopicName topicName, 
SchemaInfo schemaInfo,
+                                                      
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 = GenericJsonSchema.of(schemaInfo).getAvroSchema();
+        } catch (SchemaParseException ex) {
+            throw new PrestoException(NOT_SUPPORTED, "Topic "
+                    + topicName.toString() + " does not have a valid schema");
+        }
+
+        //TODO : check schema cyclic definitions which may case 
java.lang.StackOverflowError

Review comment:
       I wanted to leave it to the future before, **As your suggest, fix it in 
this PR may be fine choice.**
   
   Circular reference is bad schema in relational database. ( e.g. _spark_ 
https://github.com/apache/spark/blob/master/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/JavaTypeInference.scala#142).
   At current code position, Cyclic definition will case 
java.lang.StackOverflowError instead of OOM, so this only cause current query 
failed instead of JVM crash. (_so i wanted to leave it to the future before_)
   
   As your suggest , i add _StackOverflowError check_ 
(`TestAvroDecoder/TestJsonDecoder.testCyclicDefinitionDetect`) as a simple 
solution, if we need complex solution, we can add a separate PR ( based graph 
traversals: https://www.geeksforgeeks.org/detect-cycle-in-a-graph/) to optimize 
it.
   




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


Reply via email to