You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pinot.apache.org by GitBox <gi...@apache.org> on 2022/03/28 21:44:02 UTC

[GitHub] [pinot] amrishlal opened a new pull request #8426: Allow ingesting Avro complex types into JSON column.

amrishlal opened a new pull request #8426:
URL: https://github.com/apache/pinot/pull/8426


   ## Description
   This PR adds support for type conversion from avro complex types (MAP, ARRAY, RECORD, UNION) to Pinot JSON column. Currently, only avro type STRING conversion to JSON column is supported. Ingestion of complex types into JSON column can also be done using Ingestion Transform Function; however, this creates issues for us as the Pinot column name can't be of the same name as the input avro field name.)
   
   The main changes, that allows for ingesting complex types into JSON, is in `DataTypeTransformer.java` and `AvroSchemaUtil.java`. `PinotDataType.java` contains a bug fix (the fact that a simple string is also a valid json string) which allows for proper conversion of STRING field to JSON column.
   
   `AvroIngestionSchemaValidator` was modified to make sure that we no longer reject avro complex type (MAP, ARRAY, RECORD, UNION) field if they are being ingested into JSON column.
   
   The rest of the files (`PinotDataTypeTest.java`, `JsonIngestionFromAvroQueriesTest.java`, `AvroRecordExtractorTest.java`, and `AbstractRecordExtractorTest.java`) contain test cases to validate ingestion of avro complex types into JSON column.
   
   
   
   ## Upgrade Notes
   Does this PR prevent a zero down-time upgrade? (Assume upgrade order: Controller, Broker, Server, Minion)
   * [ ] Yes (Please label as **<code>backward-incompat</code>**, and complete the section below on Release Notes)
   
   Does this PR fix a zero-downtime upgrade introduced earlier?
   * [ ] Yes (Please label this as **<code>backward-incompat</code>**, and complete the section below on Release Notes)
   
   Does this PR otherwise need attention when creating release notes? Things to consider:
   - New configuration options
   - Deprecation of configurations
   - Signature changes to public methods/interfaces
   - New plugins added or old plugins removed
   * [ ] Yes (Please label this PR as **<code>release-notes</code>** and complete the section on Release Notes)
   ## Release Notes
   <!-- If you have tagged this as either backward-incompat or release-notes,
   you MUST add text here that you would like to see appear in release notes of the
   next release. -->
   
   <!-- If you have a series of commits adding or enabling a feature, then
   add this section only in final commit that marks the feature completed.
   Refer to earlier release notes to see examples of text.
   -->
   ## Documentation
   <!-- If you have introduced a new feature or configuration, please add it to the documentation as well.
   See https://docs.pinot.apache.org/developers/developers-and-contributors/update-document
   -->
   


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

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [pinot] Jackie-Jiang commented on a change in pull request #8426: Allow ingesting Avro complex types into JSON column.

Posted by GitBox <gi...@apache.org>.
Jackie-Jiang commented on a change in pull request #8426:
URL: https://github.com/apache/pinot/pull/8426#discussion_r837887248



##########
File path: pinot-core/src/test/java/org/apache/pinot/queries/JsonIngestionFromAvroQueriesTest.java
##########
@@ -0,0 +1,341 @@
+/**
+ * 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.pinot.queries;
+
+import com.google.common.collect.Lists;
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.avro.Schema;
+import org.apache.avro.file.DataFileWriter;
+import org.apache.avro.generic.GenericData;
+import org.apache.avro.generic.GenericDatumWriter;
+import org.apache.commons.io.FileUtils;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.core.common.Operator;
+import org.apache.pinot.core.operator.blocks.IntermediateResultsBlock;
+import org.apache.pinot.plugin.inputformat.avro.AvroRecordReader;
+import org.apache.pinot.segment.local.indexsegment.immutable.ImmutableSegmentLoader;
+import org.apache.pinot.segment.local.segment.creator.impl.SegmentIndexCreationDriverImpl;
+import org.apache.pinot.segment.local.segment.index.loader.IndexLoadingConfig;
+import org.apache.pinot.segment.spi.ImmutableSegment;
+import org.apache.pinot.segment.spi.IndexSegment;
+import org.apache.pinot.segment.spi.creator.SegmentGeneratorConfig;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.config.table.TableType;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.data.readers.RecordReader;
+import org.apache.pinot.spi.utils.Pair;
+import org.apache.pinot.spi.utils.ReadMode;
+import org.apache.pinot.spi.utils.builder.TableConfigBuilder;
+import org.testng.Assert;
+import org.testng.annotations.AfterClass;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+import static org.apache.avro.Schema.*;
+
+
+/**
+ * Test if ComplexType (RECORD, ARRAY, MAP, UNION, ENUM, and FIXED) field from an AVRO file can be ingested into a JSON
+ * column in a Pinot segment.
+ */
+public class JsonIngestionFromAvroQueriesTest extends BaseQueriesTest {
+  private static final File INDEX_DIR = new File(FileUtils.getTempDirectory(), "JsonIngestionFromAvroTest");
+  private static final File AVRO_DATA_FILE = new File(INDEX_DIR, "JsonIngestionFromAvroTest.avro");
+  private static final String RAW_TABLE_NAME = "testTable";
+  private static final String SEGMENT_NAME = "testSegment";
+
+  private static final String INT_COLUMN = "intColumn";
+  private static final String JSON_COLUMN_1 = "jsonColumn1"; // for testing RECORD, ARRAY, MAP, UNION
+  private static final String JSON_COLUMN_2 = "jsonColumn2"; // for testing ENUM
+  private static final String JSON_COLUMN_3 = "jsonColumn3"; // for testing FIXED
+  private static final String STRING_COLUMN = "stringColumn";
+  private static final org.apache.pinot.spi.data.Schema SCHEMA =
+      new org.apache.pinot.spi.data.Schema.SchemaBuilder().addSingleValueDimension(INT_COLUMN, FieldSpec.DataType.INT)
+          .addSingleValueDimension(JSON_COLUMN_1, FieldSpec.DataType.JSON)
+          .addSingleValueDimension(JSON_COLUMN_2, FieldSpec.DataType.JSON)
+          .addSingleValueDimension(JSON_COLUMN_3, FieldSpec.DataType.JSON)
+          .addSingleValueDimension(STRING_COLUMN, FieldSpec.DataType.STRING).build();
+  private static final TableConfig TABLE_CONFIG =
+      new TableConfigBuilder(TableType.OFFLINE).setTableName(RAW_TABLE_NAME).build();
+
+  private IndexSegment _indexSegment;
+  private List<IndexSegment> _indexSegments;
+
+  @Override
+  protected String getFilter() {
+    return "";
+  }
+
+  @Override
+  protected IndexSegment getIndexSegment() {
+    return _indexSegment;
+  }
+
+  @Override
+  protected List<IndexSegment> getIndexSegments() {
+    return _indexSegments;
+  }
+
+  /** @return {@link GenericRow} representing a row in Pinot table. */
+  private static GenericRow createTableRecord(int intValue, String stringValue, Object jsonValue,
+      GenericData.EnumSymbol enumValue, GenericData.Fixed fixedValue) {
+    GenericRow record = new GenericRow();
+    record.putValue(INT_COLUMN, intValue);
+    record.putValue(STRING_COLUMN, stringValue);
+    record.putValue(JSON_COLUMN_1, jsonValue);
+    record.putValue(JSON_COLUMN_2, enumValue);
+    record.putValue(JSON_COLUMN_3, fixedValue);
+    return record;
+  }
+
+  private static Map<String, String> createMapField(Pair<String, String>[] pairs) {
+    Map<String, String> map = new HashMap<>();
+    for (Pair<String, String> pair : pairs) {
+      map.put(pair.getFirst(), pair.getSecond());
+    }
+    return map;
+  }
+
+  private static Schema createRecordSchema() {
+    List<Field> fields = new ArrayList<>();
+    fields.add(new Field("id", create(Type.INT)));
+    fields.add(new Field("name", create(Type.STRING)));
+    return createRecord("record", "doc", JsonIngestionFromAvroQueriesTest.class.getCanonicalName(), false, fields);
+  }
+
+  private static GenericData.Record createRecordField(String k1, int v1, String k2, String v2) {
+    GenericData.Record record = new GenericData.Record(createRecordSchema());
+    record.put(k1, v1);
+    record.put(k2, v2);
+    return record;
+  }
+
+  private static GenericData.EnumSymbol createEnumField(Schema enumSchema, String enumValue) {
+    return new GenericData.EnumSymbol(enumSchema, enumValue);
+  }
+
+  private static GenericData.Fixed createFixedField(Schema fixedSchema, int value) {
+    byte[] bytes = {(byte) (value >> 24), (byte) (value >> 16), (byte) (value >> 8), (byte) value};
+    return new GenericData.Fixed(fixedSchema, bytes);
+  }
+
+  private static void createInputFile()
+      throws IOException {
+    INDEX_DIR.mkdir();
+    Schema avroSchema = Schema.createRecord("eventsRecord", null, null, false);

Review comment:
       (minor) Either remove `Schema.` or remove the static import

##########
File path: pinot-plugins/pinot-input-format/pinot-avro-base/src/main/java/org/apache/pinot/plugin/inputformat/avro/AvroSchemaUtil.java
##########
@@ -95,6 +100,9 @@ public static ObjectNode toAvroSchemaJsonObject(FieldSpec fieldSpec) {
       case BYTES:
         jsonSchema.set("type", convertStringsToJsonArray("null", "bytes"));
         return jsonSchema;
+      case JSON:
+        jsonSchema.set("type", convertStringsToJsonArray("null", "json"));

Review comment:
       Should this be `string`? I don't think avro has `json` type

##########
File path: pinot-plugins/pinot-input-format/pinot-avro-base/src/main/java/org/apache/pinot/plugin/inputformat/avro/AvroSchemaUtil.java
##########
@@ -50,6 +50,11 @@ public static DataType valueOf(Schema.Type avroType) {
         return DataType.STRING;
       case BYTES:

Review comment:
       FIXED can be handled as BYTES

##########
File path: pinot-plugins/pinot-input-format/pinot-avro-base/src/test/java/org/apache/pinot/plugin/inputformat/avro/AvroRecordExtractorTest.java
##########
@@ -73,8 +73,12 @@ protected void createInputFile()
             new Field("firstName", createUnion(Lists.newArrayList(create(Type.STRING), create(Type.NULL))), null, null),
             new Field("lastName", createUnion(Lists.newArrayList(create(Type.STRING), create(Type.NULL))), null, null),
             new Field("bids", createUnion(Lists.newArrayList(createArray(create(Type.INT)), create(Type.NULL))), null,
-                null), new Field("campaignInfo", create(Type.STRING), null, null),
-            new Field("cost", create(Type.DOUBLE), null, null), new Field("timestamp", create(Type.LONG), null, null));
+                null),
+            new Field("campaignInfo", create(Type.STRING), null, null),
+            new Field("cost", create(Type.DOUBLE), null, null),
+            new Field("timestamp", create(Type.LONG), null, null),
+            new Field("xarray", createArray(create(Type.STRING))),
+            new Field("xmap", createMap(create(Type.STRING))));

Review comment:
       Suggest adding all supported types (BOOLEAN, ENUM, FIXED, RECORD)




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

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [pinot] amrishlal commented on a change in pull request #8426: Allow ingesting Avro complex types into JSON column.

Posted by GitBox <gi...@apache.org>.
amrishlal commented on a change in pull request #8426:
URL: https://github.com/apache/pinot/pull/8426#discussion_r837997074



##########
File path: pinot-plugins/pinot-input-format/pinot-avro-base/src/test/java/org/apache/pinot/plugin/inputformat/avro/AvroRecordExtractorTest.java
##########
@@ -73,8 +73,12 @@ protected void createInputFile()
             new Field("firstName", createUnion(Lists.newArrayList(create(Type.STRING), create(Type.NULL))), null, null),
             new Field("lastName", createUnion(Lists.newArrayList(create(Type.STRING), create(Type.NULL))), null, null),
             new Field("bids", createUnion(Lists.newArrayList(createArray(create(Type.INT)), create(Type.NULL))), null,
-                null), new Field("campaignInfo", create(Type.STRING), null, null),
-            new Field("cost", create(Type.DOUBLE), null, null), new Field("timestamp", create(Type.LONG), null, null));
+                null),
+            new Field("campaignInfo", create(Type.STRING), null, null),
+            new Field("cost", create(Type.DOUBLE), null, null),
+            new Field("timestamp", create(Type.LONG), null, null),
+            new Field("xarray", createArray(create(Type.STRING))),
+            new Field("xmap", createMap(create(Type.STRING))));

Review comment:
       This file is shared by Thrift, Protobuf, ORC, CSV, etc. and I am seeing some differences in handling of boolean (Thrift vs others), fixed (custom code needed for setting Fixed values in Avro, vs binary in others), and enum (custom code needed for Avro) between different file formats, so probably a better idea to do this in a separate PR of its own.




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

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [pinot] amrishlal commented on a change in pull request #8426: Allow ingesting Avro complex types into JSON column.

Posted by GitBox <gi...@apache.org>.
amrishlal commented on a change in pull request #8426:
URL: https://github.com/apache/pinot/pull/8426#discussion_r837025385



##########
File path: pinot-core/src/test/java/org/apache/pinot/queries/JsonIngestionFromAvroQueriesTest.java
##########
@@ -0,0 +1,305 @@
+/**
+ * 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.pinot.queries;
+
+import com.google.common.collect.Lists;
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.avro.file.DataFileWriter;
+import org.apache.avro.generic.GenericData;
+import org.apache.avro.generic.GenericDatumWriter;
+import org.apache.commons.io.FileUtils;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.core.common.Operator;
+import org.apache.pinot.core.operator.blocks.IntermediateResultsBlock;
+import org.apache.pinot.plugin.inputformat.avro.AvroRecordReader;
+import org.apache.pinot.segment.local.indexsegment.immutable.ImmutableSegmentLoader;
+import org.apache.pinot.segment.local.segment.creator.impl.SegmentIndexCreationDriverImpl;
+import org.apache.pinot.segment.local.segment.index.loader.IndexLoadingConfig;
+import org.apache.pinot.segment.spi.ImmutableSegment;
+import org.apache.pinot.segment.spi.IndexSegment;
+import org.apache.pinot.segment.spi.creator.SegmentGeneratorConfig;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.config.table.TableType;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.data.readers.RecordReader;
+import org.apache.pinot.spi.utils.Pair;
+import org.apache.pinot.spi.utils.ReadMode;
+import org.apache.pinot.spi.utils.builder.TableConfigBuilder;
+import org.testng.Assert;
+import org.testng.annotations.AfterClass;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+import static org.apache.avro.Schema.*;
+
+
+/**
+ * Test if ComplexType (RECORD, ARRAY, MAP, and UNION) field from an AVRO file can be ingested into a JSON column in
+ * a Pinot segment. This class tests. Ingestion from ENUM (symbol) and FIXED (binary) is not supported.

Review comment:
       Added tests for verifying support for ingesting FIXED and ENUM into JSON column.




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

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [pinot] codecov-commenter edited a comment on pull request #8426: Allow ingesting Avro complex types into JSON column.

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #8426:
URL: https://github.com/apache/pinot/pull/8426#issuecomment-1081210576


   # [Codecov](https://codecov.io/gh/apache/pinot/pull/8426?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#8426](https://codecov.io/gh/apache/pinot/pull/8426?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (3c9ccb8) into [master](https://codecov.io/gh/apache/pinot/commit/dfc7ea8a91f0e5d1f99559f6b094d1ccc1dff5a4?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (dfc7ea8) will **decrease** coverage by `5.61%`.
   > The diff coverage is `30.76%`.
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #8426      +/-   ##
   ============================================
   - Coverage     69.66%   64.04%   -5.62%     
   - Complexity     4238     4280      +42     
   ============================================
     Files          1629     1611      -18     
     Lines         85140    84817     -323     
     Branches      12815    12875      +60     
   ============================================
   - Hits          59310    54325    -4985     
   - Misses        21698    26564    +4866     
   + Partials       4132     3928     -204     
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | integration1 | `?` | |
   | unittests1 | `66.98% <30.76%> (+0.06%)` | :arrow_up: |
   | unittests2 | `14.10% <0.00%> (-0.01%)` | :arrow_down: |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/pinot/pull/8426?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...inputformat/avro/AvroIngestionSchemaValidator.java](https://codecov.io/gh/apache/pinot/pull/8426/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtcGx1Z2lucy9waW5vdC1pbnB1dC1mb3JtYXQvcGlub3QtYXZyby1iYXNlL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9wbHVnaW4vaW5wdXRmb3JtYXQvYXZyby9BdnJvSW5nZXN0aW9uU2NoZW1hVmFsaWRhdG9yLmphdmE=) | `0.00% <0.00%> (ø)` | |
   | [.../pinot/plugin/inputformat/avro/AvroSchemaUtil.java](https://codecov.io/gh/apache/pinot/pull/8426/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtcGx1Z2lucy9waW5vdC1pbnB1dC1mb3JtYXQvcGlub3QtYXZyby1iYXNlL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9wbHVnaW4vaW5wdXRmb3JtYXQvYXZyby9BdnJvU2NoZW1hVXRpbC5qYXZh) | `17.14% <0.00%> (-1.61%)` | :arrow_down: |
   | [...a/org/apache/pinot/common/utils/PinotDataType.java](https://codecov.io/gh/apache/pinot/pull/8426/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vdXRpbHMvUGlub3REYXRhVHlwZS5qYXZh) | `81.06% <33.33%> (-0.37%)` | :arrow_down: |
   | [...t/local/recordtransformer/DataTypeTransformer.java](https://codecov.io/gh/apache/pinot/pull/8426/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC9yZWNvcmR0cmFuc2Zvcm1lci9EYXRhVHlwZVRyYW5zZm9ybWVyLmphdmE=) | `89.61% <100.00%> (+2.76%)` | :arrow_up: |
   | [...a/org/apache/pinot/common/metrics/MinionMeter.java](https://codecov.io/gh/apache/pinot/pull/8426/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vbWV0cmljcy9NaW5pb25NZXRlci5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...g/apache/pinot/common/metrics/ControllerMeter.java](https://codecov.io/gh/apache/pinot/pull/8426/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vbWV0cmljcy9Db250cm9sbGVyTWV0ZXIuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [.../apache/pinot/common/metrics/BrokerQueryPhase.java](https://codecov.io/gh/apache/pinot/pull/8426/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vbWV0cmljcy9Ccm9rZXJRdWVyeVBoYXNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [.../apache/pinot/common/metrics/MinionQueryPhase.java](https://codecov.io/gh/apache/pinot/pull/8426/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vbWV0cmljcy9NaW5pb25RdWVyeVBoYXNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...ache/pinot/server/access/AccessControlFactory.java](https://codecov.io/gh/apache/pinot/pull/8426/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VydmVyL2FjY2Vzcy9BY2Nlc3NDb250cm9sRmFjdG9yeS5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...he/pinot/common/messages/SegmentReloadMessage.java](https://codecov.io/gh/apache/pinot/pull/8426/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vbWVzc2FnZXMvU2VnbWVudFJlbG9hZE1lc3NhZ2UuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | ... and [505 more](https://codecov.io/gh/apache/pinot/pull/8426/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/pinot/pull/8426?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/pinot/pull/8426?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [dfc7ea8...3c9ccb8](https://codecov.io/gh/apache/pinot/pull/8426?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   


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

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [pinot] amrishlal commented on a change in pull request #8426: Allow ingesting Avro complex types into JSON column.

Posted by GitBox <gi...@apache.org>.
amrishlal commented on a change in pull request #8426:
URL: https://github.com/apache/pinot/pull/8426#discussion_r837995097



##########
File path: pinot-plugins/pinot-input-format/pinot-avro-base/src/main/java/org/apache/pinot/plugin/inputformat/avro/AvroSchemaUtil.java
##########
@@ -95,6 +100,9 @@ public static ObjectNode toAvroSchemaJsonObject(FieldSpec fieldSpec) {
       case BYTES:
         jsonSchema.set("type", convertStringsToJsonArray("null", "bytes"));
         return jsonSchema;
+      case JSON:
+        jsonSchema.set("type", convertStringsToJsonArray("null", "json"));

Review comment:
       Done

##########
File path: pinot-plugins/pinot-input-format/pinot-avro-base/src/main/java/org/apache/pinot/plugin/inputformat/avro/AvroSchemaUtil.java
##########
@@ -50,6 +50,11 @@ public static DataType valueOf(Schema.Type avroType) {
         return DataType.STRING;
       case BYTES:

Review comment:
       Done

##########
File path: pinot-core/src/test/java/org/apache/pinot/queries/JsonIngestionFromAvroQueriesTest.java
##########
@@ -0,0 +1,341 @@
+/**
+ * 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.pinot.queries;
+
+import com.google.common.collect.Lists;
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.avro.Schema;
+import org.apache.avro.file.DataFileWriter;
+import org.apache.avro.generic.GenericData;
+import org.apache.avro.generic.GenericDatumWriter;
+import org.apache.commons.io.FileUtils;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.core.common.Operator;
+import org.apache.pinot.core.operator.blocks.IntermediateResultsBlock;
+import org.apache.pinot.plugin.inputformat.avro.AvroRecordReader;
+import org.apache.pinot.segment.local.indexsegment.immutable.ImmutableSegmentLoader;
+import org.apache.pinot.segment.local.segment.creator.impl.SegmentIndexCreationDriverImpl;
+import org.apache.pinot.segment.local.segment.index.loader.IndexLoadingConfig;
+import org.apache.pinot.segment.spi.ImmutableSegment;
+import org.apache.pinot.segment.spi.IndexSegment;
+import org.apache.pinot.segment.spi.creator.SegmentGeneratorConfig;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.config.table.TableType;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.data.readers.RecordReader;
+import org.apache.pinot.spi.utils.Pair;
+import org.apache.pinot.spi.utils.ReadMode;
+import org.apache.pinot.spi.utils.builder.TableConfigBuilder;
+import org.testng.Assert;
+import org.testng.annotations.AfterClass;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+import static org.apache.avro.Schema.*;
+
+
+/**
+ * Test if ComplexType (RECORD, ARRAY, MAP, UNION, ENUM, and FIXED) field from an AVRO file can be ingested into a JSON
+ * column in a Pinot segment.
+ */
+public class JsonIngestionFromAvroQueriesTest extends BaseQueriesTest {
+  private static final File INDEX_DIR = new File(FileUtils.getTempDirectory(), "JsonIngestionFromAvroTest");
+  private static final File AVRO_DATA_FILE = new File(INDEX_DIR, "JsonIngestionFromAvroTest.avro");
+  private static final String RAW_TABLE_NAME = "testTable";
+  private static final String SEGMENT_NAME = "testSegment";
+
+  private static final String INT_COLUMN = "intColumn";
+  private static final String JSON_COLUMN_1 = "jsonColumn1"; // for testing RECORD, ARRAY, MAP, UNION
+  private static final String JSON_COLUMN_2 = "jsonColumn2"; // for testing ENUM
+  private static final String JSON_COLUMN_3 = "jsonColumn3"; // for testing FIXED
+  private static final String STRING_COLUMN = "stringColumn";
+  private static final org.apache.pinot.spi.data.Schema SCHEMA =
+      new org.apache.pinot.spi.data.Schema.SchemaBuilder().addSingleValueDimension(INT_COLUMN, FieldSpec.DataType.INT)
+          .addSingleValueDimension(JSON_COLUMN_1, FieldSpec.DataType.JSON)
+          .addSingleValueDimension(JSON_COLUMN_2, FieldSpec.DataType.JSON)
+          .addSingleValueDimension(JSON_COLUMN_3, FieldSpec.DataType.JSON)
+          .addSingleValueDimension(STRING_COLUMN, FieldSpec.DataType.STRING).build();
+  private static final TableConfig TABLE_CONFIG =
+      new TableConfigBuilder(TableType.OFFLINE).setTableName(RAW_TABLE_NAME).build();
+
+  private IndexSegment _indexSegment;
+  private List<IndexSegment> _indexSegments;
+
+  @Override
+  protected String getFilter() {
+    return "";
+  }
+
+  @Override
+  protected IndexSegment getIndexSegment() {
+    return _indexSegment;
+  }
+
+  @Override
+  protected List<IndexSegment> getIndexSegments() {
+    return _indexSegments;
+  }
+
+  /** @return {@link GenericRow} representing a row in Pinot table. */
+  private static GenericRow createTableRecord(int intValue, String stringValue, Object jsonValue,
+      GenericData.EnumSymbol enumValue, GenericData.Fixed fixedValue) {
+    GenericRow record = new GenericRow();
+    record.putValue(INT_COLUMN, intValue);
+    record.putValue(STRING_COLUMN, stringValue);
+    record.putValue(JSON_COLUMN_1, jsonValue);
+    record.putValue(JSON_COLUMN_2, enumValue);
+    record.putValue(JSON_COLUMN_3, fixedValue);
+    return record;
+  }
+
+  private static Map<String, String> createMapField(Pair<String, String>[] pairs) {
+    Map<String, String> map = new HashMap<>();
+    for (Pair<String, String> pair : pairs) {
+      map.put(pair.getFirst(), pair.getSecond());
+    }
+    return map;
+  }
+
+  private static Schema createRecordSchema() {
+    List<Field> fields = new ArrayList<>();
+    fields.add(new Field("id", create(Type.INT)));
+    fields.add(new Field("name", create(Type.STRING)));
+    return createRecord("record", "doc", JsonIngestionFromAvroQueriesTest.class.getCanonicalName(), false, fields);
+  }
+
+  private static GenericData.Record createRecordField(String k1, int v1, String k2, String v2) {
+    GenericData.Record record = new GenericData.Record(createRecordSchema());
+    record.put(k1, v1);
+    record.put(k2, v2);
+    return record;
+  }
+
+  private static GenericData.EnumSymbol createEnumField(Schema enumSchema, String enumValue) {
+    return new GenericData.EnumSymbol(enumSchema, enumValue);
+  }
+
+  private static GenericData.Fixed createFixedField(Schema fixedSchema, int value) {
+    byte[] bytes = {(byte) (value >> 24), (byte) (value >> 16), (byte) (value >> 8), (byte) value};
+    return new GenericData.Fixed(fixedSchema, bytes);
+  }
+
+  private static void createInputFile()
+      throws IOException {
+    INDEX_DIR.mkdir();
+    Schema avroSchema = Schema.createRecord("eventsRecord", null, null, false);

Review comment:
       Done




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

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [pinot] codecov-commenter edited a comment on pull request #8426: Allow ingesting Avro complex types into JSON column.

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #8426:
URL: https://github.com/apache/pinot/pull/8426#issuecomment-1081210576


   # [Codecov](https://codecov.io/gh/apache/pinot/pull/8426?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#8426](https://codecov.io/gh/apache/pinot/pull/8426?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (48b300a) into [master](https://codecov.io/gh/apache/pinot/commit/dfc7ea8a91f0e5d1f99559f6b094d1ccc1dff5a4?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (dfc7ea8) will **increase** coverage by `1.26%`.
   > The diff coverage is `36.36%`.
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #8426      +/-   ##
   ============================================
   + Coverage     69.66%   70.92%   +1.26%     
   - Complexity     4238     4280      +42     
   ============================================
     Files          1629     1660      +31     
     Lines         85140    87039    +1899     
     Branches      12815    13140     +325     
   ============================================
   + Hits          59310    61734    +2424     
   + Misses        21698    21044     -654     
   - Partials       4132     4261     +129     
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | integration1 | `28.73% <0.00%> (-0.06%)` | :arrow_down: |
   | integration2 | `27.37% <0.00%> (?)` | |
   | unittests1 | `66.99% <36.36%> (+0.07%)` | :arrow_up: |
   | unittests2 | `14.19% <0.00%> (+0.08%)` | :arrow_up: |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/pinot/pull/8426?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...inputformat/avro/AvroIngestionSchemaValidator.java](https://codecov.io/gh/apache/pinot/pull/8426/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtcGx1Z2lucy9waW5vdC1pbnB1dC1mb3JtYXQvcGlub3QtYXZyby1iYXNlL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9wbHVnaW4vaW5wdXRmb3JtYXQvYXZyby9BdnJvSW5nZXN0aW9uU2NoZW1hVmFsaWRhdG9yLmphdmE=) | `0.00% <0.00%> (ø)` | |
   | [.../pinot/plugin/inputformat/avro/AvroSchemaUtil.java](https://codecov.io/gh/apache/pinot/pull/8426/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtcGx1Z2lucy9waW5vdC1pbnB1dC1mb3JtYXQvcGlub3QtYXZyby1iYXNlL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9wbHVnaW4vaW5wdXRmb3JtYXQvYXZyby9BdnJvU2NoZW1hVXRpbC5qYXZh) | `18.18% <0.00%> (-0.57%)` | :arrow_down: |
   | [...a/org/apache/pinot/common/utils/PinotDataType.java](https://codecov.io/gh/apache/pinot/pull/8426/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vdXRpbHMvUGlub3REYXRhVHlwZS5qYXZh) | `81.06% <33.33%> (-0.37%)` | :arrow_down: |
   | [...t/local/recordtransformer/DataTypeTransformer.java](https://codecov.io/gh/apache/pinot/pull/8426/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC9yZWNvcmR0cmFuc2Zvcm1lci9EYXRhVHlwZVRyYW5zZm9ybWVyLmphdmE=) | `89.61% <100.00%> (+2.76%)` | :arrow_up: |
   | [...he/pinot/core/plan/AggregationGroupByPlanNode.java](https://codecov.io/gh/apache/pinot/pull/8426/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9wbGFuL0FnZ3JlZ2F0aW9uR3JvdXBCeVBsYW5Ob2RlLmphdmE=) | `45.45% <0.00%> (-36.37%)` | :arrow_down: |
   | [...org/apache/pinot/spi/config/table/QueryConfig.java](https://codecov.io/gh/apache/pinot/pull/8426/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3Qtc3BpL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9zcGkvY29uZmlnL3RhYmxlL1F1ZXJ5Q29uZmlnLmphdmE=) | `53.33% <0.00%> (-32.39%)` | :arrow_down: |
   | [...ansformer/datetime/DateTimeTransformerFactory.java](https://codecov.io/gh/apache/pinot/pull/8426/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9vcGVyYXRvci90cmFuc2Zvcm0vdHJhbnNmb3JtZXIvZGF0ZXRpbWUvRGF0ZVRpbWVUcmFuc2Zvcm1lckZhY3RvcnkuamF2YQ==) | `75.00% <0.00%> (-25.00%)` | :arrow_down: |
   | [.../pinot/common/function/DateTimePatternHandler.java](https://codecov.io/gh/apache/pinot/pull/8426/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vZnVuY3Rpb24vRGF0ZVRpbWVQYXR0ZXJuSGFuZGxlci5qYXZh) | `80.00% <0.00%> (-20.00%)` | :arrow_down: |
   | [...ment/virtualcolumn/DocIdVirtualColumnProvider.java](https://codecov.io/gh/apache/pinot/pull/8426/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC9zZWdtZW50L3ZpcnR1YWxjb2x1bW4vRG9jSWRWaXJ0dWFsQ29sdW1uUHJvdmlkZXIuamF2YQ==) | `76.92% <0.00%> (-15.39%)` | :arrow_down: |
   | [.../apache/pinot/common/function/FunctionInvoker.java](https://codecov.io/gh/apache/pinot/pull/8426/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vZnVuY3Rpb24vRnVuY3Rpb25JbnZva2VyLmphdmE=) | `79.54% <0.00%> (-11.37%)` | :arrow_down: |
   | ... and [301 more](https://codecov.io/gh/apache/pinot/pull/8426/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/pinot/pull/8426?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/pinot/pull/8426?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [dfc7ea8...48b300a](https://codecov.io/gh/apache/pinot/pull/8426?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   


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

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [pinot] codecov-commenter edited a comment on pull request #8426: Allow ingesting Avro complex types into JSON column.

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #8426:
URL: https://github.com/apache/pinot/pull/8426#issuecomment-1081210576


   # [Codecov](https://codecov.io/gh/apache/pinot/pull/8426?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#8426](https://codecov.io/gh/apache/pinot/pull/8426?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (475fd49) into [master](https://codecov.io/gh/apache/pinot/commit/dfc7ea8a91f0e5d1f99559f6b094d1ccc1dff5a4?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (dfc7ea8) will **decrease** coverage by `42.33%`.
   > The diff coverage is `44.38%`.
   
   > :exclamation: Current head 475fd49 differs from pull request most recent head f8773fd. Consider uploading reports for the commit f8773fd to get more accurate results
   
   ```diff
   @@              Coverage Diff              @@
   ##             master    #8426       +/-   ##
   =============================================
   - Coverage     69.66%   27.32%   -42.34%     
   =============================================
     Files          1629     1644       +15     
     Lines         85140    86343     +1203     
     Branches      12815    13040      +225     
   =============================================
   - Hits          59310    23593    -35717     
   - Misses        21698    60528    +38830     
   + Partials       4132     2222     -1910     
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | integration1 | `?` | |
   | integration2 | `27.32% <44.38%> (?)` | |
   | unittests1 | `?` | |
   | unittests2 | `?` | |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/pinot/pull/8426?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...e/pinot/broker/api/resources/PinotBrokerDebug.java](https://codecov.io/gh/apache/pinot/pull/8426/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtYnJva2VyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9icm9rZXIvYXBpL3Jlc291cmNlcy9QaW5vdEJyb2tlckRlYnVnLmphdmE=) | `76.47% <ø> (ø)` | |
   | [...pinot/broker/api/resources/PinotBrokerRouting.java](https://codecov.io/gh/apache/pinot/pull/8426/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtYnJva2VyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9icm9rZXIvYXBpL3Jlc291cmNlcy9QaW5vdEJyb2tlclJvdXRpbmcuamF2YQ==) | `0.00% <ø> (ø)` | |
   | [.../BrokerResourceOnlineOfflineStateModelFactory.java](https://codecov.io/gh/apache/pinot/pull/8426/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtYnJva2VyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9icm9rZXIvYnJva2VyL2hlbGl4L0Jyb2tlclJlc291cmNlT25saW5lT2ZmbGluZVN0YXRlTW9kZWxGYWN0b3J5LmphdmE=) | `55.81% <ø> (ø)` | |
   | [.../helix/BrokerUserDefinedMessageHandlerFactory.java](https://codecov.io/gh/apache/pinot/pull/8426/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtYnJva2VyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9icm9rZXIvYnJva2VyL2hlbGl4L0Jyb2tlclVzZXJEZWZpbmVkTWVzc2FnZUhhbmRsZXJGYWN0b3J5LmphdmE=) | `52.83% <ø> (-16.99%)` | :arrow_down: |
   | [.../routing/segmentpruner/PartitionSegmentPruner.java](https://codecov.io/gh/apache/pinot/pull/8426/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtYnJva2VyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9icm9rZXIvcm91dGluZy9zZWdtZW50cHJ1bmVyL1BhcnRpdGlvblNlZ21lbnRQcnVuZXIuamF2YQ==) | `0.00% <0.00%> (-67.97%)` | :arrow_down: |
   | [...roker/routing/segmentpruner/TimeSegmentPruner.java](https://codecov.io/gh/apache/pinot/pull/8426/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtYnJva2VyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9icm9rZXIvcm91dGluZy9zZWdtZW50cHJ1bmVyL1RpbWVTZWdtZW50UHJ1bmVyLmphdmE=) | `0.00% <0.00%> (-80.88%)` | :arrow_down: |
   | [...inot/common/function/scalar/DateTimeFunctions.java](https://codecov.io/gh/apache/pinot/pull/8426/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vZnVuY3Rpb24vc2NhbGFyL0RhdGVUaW1lRnVuY3Rpb25zLmphdmE=) | `5.05% <0.00%> (-93.58%)` | :arrow_down: |
   | [...he/pinot/common/function/scalar/JsonFunctions.java](https://codecov.io/gh/apache/pinot/pull/8426/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vZnVuY3Rpb24vc2NhbGFyL0pzb25GdW5jdGlvbnMuamF2YQ==) | `25.75% <0.00%> (-62.13%)` | :arrow_down: |
   | [...a/org/apache/pinot/common/metrics/ServerGauge.java](https://codecov.io/gh/apache/pinot/pull/8426/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vbWV0cmljcy9TZXJ2ZXJHYXVnZS5qYXZh) | `95.65% <ø> (-0.19%)` | :arrow_down: |
   | [...he/pinot/common/request/context/FilterContext.java](https://codecov.io/gh/apache/pinot/pull/8426/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vcmVxdWVzdC9jb250ZXh0L0ZpbHRlckNvbnRleHQuamF2YQ==) | `41.02% <0.00%> (-37.36%)` | :arrow_down: |
   | ... and [1383 more](https://codecov.io/gh/apache/pinot/pull/8426/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/pinot/pull/8426?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/pinot/pull/8426?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [dfc7ea8...f8773fd](https://codecov.io/gh/apache/pinot/pull/8426?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   


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

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [pinot] codecov-commenter edited a comment on pull request #8426: Allow ingesting Avro complex types into JSON column.

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #8426:
URL: https://github.com/apache/pinot/pull/8426#issuecomment-1081210576


   # [Codecov](https://codecov.io/gh/apache/pinot/pull/8426?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#8426](https://codecov.io/gh/apache/pinot/pull/8426?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (d338777) into [master](https://codecov.io/gh/apache/pinot/commit/dfc7ea8a91f0e5d1f99559f6b094d1ccc1dff5a4?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (dfc7ea8) will **decrease** coverage by `5.47%`.
   > The diff coverage is `36.36%`.
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #8426      +/-   ##
   ============================================
   - Coverage     69.66%   64.18%   -5.48%     
   - Complexity     4238     4281      +43     
   ============================================
     Files          1629     1615      -14     
     Lines         85140    85158      +18     
     Branches      12815    12937     +122     
   ============================================
   - Hits          59310    54662    -4648     
   - Misses        21698    26538    +4840     
   + Partials       4132     3958     -174     
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | integration1 | `?` | |
   | unittests1 | `67.01% <36.36%> (+0.08%)` | :arrow_up: |
   | unittests2 | `14.20% <0.00%> (+0.08%)` | :arrow_up: |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/pinot/pull/8426?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...inputformat/avro/AvroIngestionSchemaValidator.java](https://codecov.io/gh/apache/pinot/pull/8426/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtcGx1Z2lucy9waW5vdC1pbnB1dC1mb3JtYXQvcGlub3QtYXZyby1iYXNlL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9wbHVnaW4vaW5wdXRmb3JtYXQvYXZyby9BdnJvSW5nZXN0aW9uU2NoZW1hVmFsaWRhdG9yLmphdmE=) | `0.00% <0.00%> (ø)` | |
   | [.../pinot/plugin/inputformat/avro/AvroSchemaUtil.java](https://codecov.io/gh/apache/pinot/pull/8426/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtcGx1Z2lucy9waW5vdC1pbnB1dC1mb3JtYXQvcGlub3QtYXZyby1iYXNlL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9wbHVnaW4vaW5wdXRmb3JtYXQvYXZyby9BdnJvU2NoZW1hVXRpbC5qYXZh) | `18.18% <0.00%> (-0.57%)` | :arrow_down: |
   | [...a/org/apache/pinot/common/utils/PinotDataType.java](https://codecov.io/gh/apache/pinot/pull/8426/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vdXRpbHMvUGlub3REYXRhVHlwZS5qYXZh) | `81.06% <33.33%> (-0.37%)` | :arrow_down: |
   | [...t/local/recordtransformer/DataTypeTransformer.java](https://codecov.io/gh/apache/pinot/pull/8426/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC9yZWNvcmR0cmFuc2Zvcm1lci9EYXRhVHlwZVRyYW5zZm9ybWVyLmphdmE=) | `89.61% <100.00%> (+2.76%)` | :arrow_up: |
   | [...a/org/apache/pinot/common/metrics/MinionMeter.java](https://codecov.io/gh/apache/pinot/pull/8426/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vbWV0cmljcy9NaW5pb25NZXRlci5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...g/apache/pinot/common/metrics/ControllerMeter.java](https://codecov.io/gh/apache/pinot/pull/8426/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vbWV0cmljcy9Db250cm9sbGVyTWV0ZXIuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [.../apache/pinot/common/metrics/BrokerQueryPhase.java](https://codecov.io/gh/apache/pinot/pull/8426/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vbWV0cmljcy9Ccm9rZXJRdWVyeVBoYXNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [.../apache/pinot/common/metrics/MinionQueryPhase.java](https://codecov.io/gh/apache/pinot/pull/8426/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vbWV0cmljcy9NaW5pb25RdWVyeVBoYXNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...ache/pinot/server/access/AccessControlFactory.java](https://codecov.io/gh/apache/pinot/pull/8426/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VydmVyL2FjY2Vzcy9BY2Nlc3NDb250cm9sRmFjdG9yeS5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...he/pinot/common/messages/SegmentReloadMessage.java](https://codecov.io/gh/apache/pinot/pull/8426/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vbWVzc2FnZXMvU2VnbWVudFJlbG9hZE1lc3NhZ2UuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | ... and [512 more](https://codecov.io/gh/apache/pinot/pull/8426/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/pinot/pull/8426?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/pinot/pull/8426?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [dfc7ea8...d338777](https://codecov.io/gh/apache/pinot/pull/8426?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   


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

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [pinot] Jackie-Jiang commented on a change in pull request #8426: Allow ingesting Avro complex types into JSON column.

Posted by GitBox <gi...@apache.org>.
Jackie-Jiang commented on a change in pull request #8426:
URL: https://github.com/apache/pinot/pull/8426#discussion_r836885719



##########
File path: pinot-core/src/test/java/org/apache/pinot/queries/JsonIngestionFromAvroQueriesTest.java
##########
@@ -0,0 +1,305 @@
+/**
+ * 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.pinot.queries;
+
+import com.google.common.collect.Lists;
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.avro.file.DataFileWriter;
+import org.apache.avro.generic.GenericData;
+import org.apache.avro.generic.GenericDatumWriter;
+import org.apache.commons.io.FileUtils;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.core.common.Operator;
+import org.apache.pinot.core.operator.blocks.IntermediateResultsBlock;
+import org.apache.pinot.plugin.inputformat.avro.AvroRecordReader;
+import org.apache.pinot.segment.local.indexsegment.immutable.ImmutableSegmentLoader;
+import org.apache.pinot.segment.local.segment.creator.impl.SegmentIndexCreationDriverImpl;
+import org.apache.pinot.segment.local.segment.index.loader.IndexLoadingConfig;
+import org.apache.pinot.segment.spi.ImmutableSegment;
+import org.apache.pinot.segment.spi.IndexSegment;
+import org.apache.pinot.segment.spi.creator.SegmentGeneratorConfig;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.config.table.TableType;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.data.readers.RecordReader;
+import org.apache.pinot.spi.utils.Pair;
+import org.apache.pinot.spi.utils.ReadMode;
+import org.apache.pinot.spi.utils.builder.TableConfigBuilder;
+import org.testng.Assert;
+import org.testng.annotations.AfterClass;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+import static org.apache.avro.Schema.*;
+
+
+/**
+ * Test if ComplexType (RECORD, ARRAY, MAP, and UNION) field from an AVRO file can be ingested into a JSON column in
+ * a Pinot segment. This class tests. Ingestion from ENUM (symbol) and FIXED (binary) is not supported.

Review comment:
       This class tests ...

##########
File path: pinot-common/src/main/java/org/apache/pinot/common/utils/PinotDataType.java
##########
@@ -829,18 +829,23 @@ public String toString(Object value) {
   public String toJson(Object value) {
     if (value instanceof String) {
       try {
+        // Try to parse the string as JSON first
         return JsonUtils.stringToJsonNode((String) value).toString();
+      } catch (com.fasterxml.jackson.core.JsonParseException jpe) {

Review comment:
       Import this class

##########
File path: pinot-core/src/test/java/org/apache/pinot/queries/JsonIngestionFromAvroQueriesTest.java
##########
@@ -0,0 +1,305 @@
+/**
+ * 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.pinot.queries;
+
+import com.google.common.collect.Lists;
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.avro.file.DataFileWriter;
+import org.apache.avro.generic.GenericData;
+import org.apache.avro.generic.GenericDatumWriter;
+import org.apache.commons.io.FileUtils;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.core.common.Operator;
+import org.apache.pinot.core.operator.blocks.IntermediateResultsBlock;
+import org.apache.pinot.plugin.inputformat.avro.AvroRecordReader;
+import org.apache.pinot.segment.local.indexsegment.immutable.ImmutableSegmentLoader;
+import org.apache.pinot.segment.local.segment.creator.impl.SegmentIndexCreationDriverImpl;
+import org.apache.pinot.segment.local.segment.index.loader.IndexLoadingConfig;
+import org.apache.pinot.segment.spi.ImmutableSegment;
+import org.apache.pinot.segment.spi.IndexSegment;
+import org.apache.pinot.segment.spi.creator.SegmentGeneratorConfig;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.config.table.TableType;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.data.readers.RecordReader;
+import org.apache.pinot.spi.utils.Pair;
+import org.apache.pinot.spi.utils.ReadMode;
+import org.apache.pinot.spi.utils.builder.TableConfigBuilder;
+import org.testng.Assert;
+import org.testng.annotations.AfterClass;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+import static org.apache.avro.Schema.*;
+
+
+/**
+ * Test if ComplexType (RECORD, ARRAY, MAP, and UNION) field from an AVRO file can be ingested into a JSON column in
+ * a Pinot segment. This class tests. Ingestion from ENUM (symbol) and FIXED (binary) is not supported.

Review comment:
       IIRC, `ENUM` is already extracted as `STRING` data type; `FIXED` can be extracted as `BYTES` data type. Since we are working on this, I'd suggest support all the avro data types.

##########
File path: pinot-core/src/test/java/org/apache/pinot/queries/JsonIngestionFromAvroQueriesTest.java
##########
@@ -0,0 +1,305 @@
+/**
+ * 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.pinot.queries;
+
+import com.google.common.collect.Lists;
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.avro.file.DataFileWriter;
+import org.apache.avro.generic.GenericData;
+import org.apache.avro.generic.GenericDatumWriter;
+import org.apache.commons.io.FileUtils;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.core.common.Operator;
+import org.apache.pinot.core.operator.blocks.IntermediateResultsBlock;
+import org.apache.pinot.plugin.inputformat.avro.AvroRecordReader;
+import org.apache.pinot.segment.local.indexsegment.immutable.ImmutableSegmentLoader;
+import org.apache.pinot.segment.local.segment.creator.impl.SegmentIndexCreationDriverImpl;
+import org.apache.pinot.segment.local.segment.index.loader.IndexLoadingConfig;
+import org.apache.pinot.segment.spi.ImmutableSegment;
+import org.apache.pinot.segment.spi.IndexSegment;
+import org.apache.pinot.segment.spi.creator.SegmentGeneratorConfig;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.config.table.TableType;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.data.readers.RecordReader;
+import org.apache.pinot.spi.utils.Pair;
+import org.apache.pinot.spi.utils.ReadMode;
+import org.apache.pinot.spi.utils.builder.TableConfigBuilder;
+import org.testng.Assert;
+import org.testng.annotations.AfterClass;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+import static org.apache.avro.Schema.*;

Review comment:
       Seems there are more usage of avro Schema than pinot Schema. Consider import avro Schema and use full path for pinot Schema

##########
File path: pinot-core/src/test/java/org/apache/pinot/queries/JsonIngestionFromAvroQueriesTest.java
##########
@@ -0,0 +1,305 @@
+/**
+ * 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.pinot.queries;
+
+import com.google.common.collect.Lists;
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.avro.file.DataFileWriter;
+import org.apache.avro.generic.GenericData;
+import org.apache.avro.generic.GenericDatumWriter;
+import org.apache.commons.io.FileUtils;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.core.common.Operator;
+import org.apache.pinot.core.operator.blocks.IntermediateResultsBlock;
+import org.apache.pinot.plugin.inputformat.avro.AvroRecordReader;
+import org.apache.pinot.segment.local.indexsegment.immutable.ImmutableSegmentLoader;
+import org.apache.pinot.segment.local.segment.creator.impl.SegmentIndexCreationDriverImpl;
+import org.apache.pinot.segment.local.segment.index.loader.IndexLoadingConfig;
+import org.apache.pinot.segment.spi.ImmutableSegment;
+import org.apache.pinot.segment.spi.IndexSegment;
+import org.apache.pinot.segment.spi.creator.SegmentGeneratorConfig;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.config.table.TableType;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.data.readers.RecordReader;
+import org.apache.pinot.spi.utils.Pair;
+import org.apache.pinot.spi.utils.ReadMode;
+import org.apache.pinot.spi.utils.builder.TableConfigBuilder;
+import org.testng.Assert;
+import org.testng.annotations.AfterClass;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+import static org.apache.avro.Schema.*;
+
+
+/**
+ * Test if ComplexType (RECORD, ARRAY, MAP, and UNION) field from an AVRO file can be ingested into a JSON column in
+ * a Pinot segment. This class tests. Ingestion from ENUM (symbol) and FIXED (binary) is not supported.
+ */
+public class JsonIngestionFromAvroQueriesTest extends BaseQueriesTest {
+  private static final File INDEX_DIR = new File(FileUtils.getTempDirectory(), "JsonIngestionFromAvroTest");
+  private static final File AVRO_DATA_FILE = new File(INDEX_DIR, "JsonIngestionFromAvroTest.avro");
+  private static final String RAW_TABLE_NAME = "testTable";
+  private static final String SEGMENT_NAME = "testSegment";
+
+  private static final String INT_COLUMN = "intColumn";
+  private static final String JSON_COLUMN = "jsonColumn";
+  private static final String STRING_COLUMN = "stringColumn";
+  private static final Schema SCHEMA =
+      new Schema.SchemaBuilder().addSingleValueDimension(INT_COLUMN, FieldSpec.DataType.INT)
+          .addSingleValueDimension(JSON_COLUMN, FieldSpec.DataType.JSON)
+          .addSingleValueDimension(STRING_COLUMN, FieldSpec.DataType.STRING).build();
+  private static final TableConfig TABLE_CONFIG =
+      new TableConfigBuilder(TableType.OFFLINE).setTableName(RAW_TABLE_NAME).build();
+
+  private IndexSegment _indexSegment;
+  private List<IndexSegment> _indexSegments;
+
+  @Override
+  protected String getFilter() {
+    return "";
+  }
+
+  @Override
+  protected IndexSegment getIndexSegment() {
+    return _indexSegment;
+  }
+
+  @Override
+  protected List<IndexSegment> getIndexSegments() {
+    return _indexSegments;
+  }
+
+  /** @return {@link GenericRow} representing a row in Pinot table. */
+  private static GenericRow createTableRecord(int intValue, String stringValue, Object jsonValue) {
+    GenericRow record = new GenericRow();
+    record.putValue(INT_COLUMN, intValue);
+    record.putValue(STRING_COLUMN, stringValue);
+    record.putValue(JSON_COLUMN, jsonValue);
+
+    return record;
+  }
+
+  private static Map<String, String> createMapField(Pair<String, String>[] pairs) {
+    Map<String, String> map = new HashMap<>();
+    for (Pair<String, String> pair : pairs) {
+      map.put(pair.getFirst(), pair.getSecond());
+    }
+    return map;
+  }
+
+  private static org.apache.avro.Schema createRecordSchema() {
+    List<org.apache.avro.Schema.Field> fields = new ArrayList<>();
+    fields.add(new org.apache.avro.Schema.Field("id", create(Type.INT)));
+    fields.add(new org.apache.avro.Schema.Field("name", create(Type.STRING)));
+    return createRecord("record", "doc", JsonIngestionFromAvroQueriesTest.class.getCanonicalName(), false, fields);
+  }
+
+  private static GenericData.Record createRecordField(String k1, int v1, String k2, String v2) {
+    GenericData.Record record = new GenericData.Record(createRecordSchema());
+    record.put(k1, v1);
+    record.put(k2, v2);
+    return record;
+  }
+
+  private static void createInputFile()
+      throws IOException {
+    INDEX_DIR.mkdir();
+    org.apache.avro.Schema avroSchema = org.apache.avro.Schema.createRecord("eventsRecord", null, null, false);
+    List<Field> fields = Arrays
+        .asList(new Field(INT_COLUMN, createUnion(Lists.newArrayList(create(Type.INT), create(Type.NULL))), null, null),
+            new Field(STRING_COLUMN, createUnion(Lists.newArrayList(create(Type.STRING), create(Type.NULL))), null,
+                null), new Field(JSON_COLUMN,
+                createUnion(createArray(create(Type.STRING)), createMap(create(Type.STRING)), createRecordSchema(),
+                    create(Type.STRING), create(Type.NULL))));
+    avroSchema.setFields(fields);
+    List<GenericRow> inputRecords = new ArrayList<>();
+    // Insert ARRAY
+    inputRecords.add(createTableRecord(1, "daffy duck", Arrays.asList("this", "is", "a", "test")));
+
+    // Insert MAP
+    inputRecords
+        .add(createTableRecord(2, "mickey mouse", createMapField(new Pair[]{new Pair("a", "1"), new Pair("b", "2")})));
+    inputRecords
+        .add(createTableRecord(3, "donald duck", createMapField(new Pair[]{new Pair("a", "1"), new Pair("b", "2")})));
+    inputRecords.add(
+        createTableRecord(4, "scrooge mcduck", createMapField(new Pair[]{new Pair("a", "1"), new Pair("b", "2")})));
+
+    // insert RECORD
+    inputRecords.add(createTableRecord(5, "minney mouse", createRecordField("id", 1, "name", "minney")));
+
+    // Insert simple Java String (gets converted into JSON value)
+    inputRecords.add(createTableRecord(6, "pluto", "test"));
+
+    // Insert JSON string (gets converted into JSON document)
+    inputRecords.add(createTableRecord(7, "scooby doo", "{\"name\":\"scooby\",\"id\":7}"));
+
+    try (DataFileWriter<GenericData.Record> fileWriter = new DataFileWriter<>(new GenericDatumWriter<>(avroSchema))) {
+      fileWriter.create(avroSchema, AVRO_DATA_FILE);
+      for (GenericRow inputRecord : inputRecords) {
+        GenericData.Record record = new GenericData.Record(avroSchema);
+        record.put(INT_COLUMN, inputRecord.getValue(INT_COLUMN));
+        record.put(STRING_COLUMN, inputRecord.getValue(STRING_COLUMN));
+        record.put(JSON_COLUMN, inputRecord.getValue(JSON_COLUMN));
+        fileWriter.append(record);
+      }
+    }
+  }
+
+  private static RecordReader createRecordReader()
+      throws IOException {
+    Set<String> set = new HashSet<>();
+    set.add(INT_COLUMN);
+    set.add(STRING_COLUMN);
+    set.add(JSON_COLUMN);
+    AvroRecordReader avroRecordReader = new AvroRecordReader();
+    avroRecordReader.init(AVRO_DATA_FILE, set, null);
+    return avroRecordReader;
+  }
+
+  /** Create an AVRO file and then ingest it into Pinot while creating a JsonIndex. */
+  @BeforeClass
+  public void setUp()
+      throws Exception {
+    FileUtils.deleteDirectory(INDEX_DIR);
+    createInputFile();
+
+    List<String> jsonIndexColumns = new ArrayList<>();
+    jsonIndexColumns.add("jsonColumn");
+    TABLE_CONFIG.getIndexingConfig().setJsonIndexColumns(jsonIndexColumns);
+    SegmentGeneratorConfig segmentGeneratorConfig = new SegmentGeneratorConfig(TABLE_CONFIG, SCHEMA);
+    segmentGeneratorConfig.setTableName(RAW_TABLE_NAME);
+    segmentGeneratorConfig.setSegmentName(SEGMENT_NAME);
+    segmentGeneratorConfig.setOutDir(INDEX_DIR.getPath());
+    segmentGeneratorConfig.setInputFilePath(AVRO_DATA_FILE.getPath());
+
+    SegmentIndexCreationDriverImpl driver = new SegmentIndexCreationDriverImpl();
+    driver.init(segmentGeneratorConfig, createRecordReader());
+    driver.build();
+
+    IndexLoadingConfig indexLoadingConfig = new IndexLoadingConfig();
+    indexLoadingConfig.setTableConfig(TABLE_CONFIG);
+    indexLoadingConfig.setJsonIndexColumns(new HashSet<String>(jsonIndexColumns));
+    indexLoadingConfig.setReadMode(ReadMode.mmap);
+
+    ImmutableSegment immutableSegment =
+        ImmutableSegmentLoader.load(new File(INDEX_DIR, SEGMENT_NAME), indexLoadingConfig);
+    _indexSegment = immutableSegment;
+    _indexSegments = Arrays.asList(immutableSegment, immutableSegment);
+  }
+
+  /** Verify that we can query the JSON column that ingested ComplexType data from an AVRO file (see setUp). */
+  @Test
+  public void testSimpleSelectOnJsonColumn() {
+    try {
+      Operator operator = getOperatorForSqlQuery("select intColumn, stringColumn, jsonColumn FROM testTable limit 100");
+      IntermediateResultsBlock block = (IntermediateResultsBlock) operator.nextBlock();
+      Collection<Object[]> rows = block.getSelectionResult();
+      Assert.assertEquals(block.getDataSchema().getColumnDataType(0), DataSchema.ColumnDataType.INT);
+      Assert.assertEquals(block.getDataSchema().getColumnDataType(1), DataSchema.ColumnDataType.STRING);
+      Assert.assertEquals(block.getDataSchema().getColumnDataType(2), DataSchema.ColumnDataType.JSON);
+
+      List<String> expecteds = Arrays
+          .asList("[1, daffy duck, [\"this\",\"is\",\"a\",\"test\"]]", "[2, mickey mouse, {\"a\":\"1\",\"b\":\"2\"}]",
+              "[3, donald duck, {\"a\":\"1\",\"b\":\"2\"}]", "[4, scrooge mcduck, {\"a\":\"1\",\"b\":\"2\"}]",
+              "[5, minney mouse, {\"name\":\"minney\",\"id\":1}]", "[6, pluto, \"test\"]",
+              "[7, scooby doo, {\"name\":\"scooby\",\"id\":7}]");
+      int index = 0;
+
+      Iterator<Object[]> iterator = rows.iterator();
+      while (iterator.hasNext()) {
+        Object[] row = iterator.next();
+        System.out.println(Arrays.toString(row));
+        Assert.assertEquals(Arrays.toString(row), expecteds.get(index++));
+      }
+    } catch (IllegalStateException ise) {
+      Assert.assertTrue(true);
+    }
+  }
+
+  /** Verify simple path expression query on ingested Avro file. */
+  @Test
+  public void testJsonPathSelectOnJsonColumn() {
+    try {
+      Operator operator = getOperatorForSqlQuery("select intColumn, json_extract_scalar(jsonColumn, '$.name', "
+          + "'STRING', 'null') FROM testTable");
+      IntermediateResultsBlock block = (IntermediateResultsBlock) operator.nextBlock();
+      Collection<Object[]> rows = block.getSelectionResult();
+      Assert.assertEquals(block.getDataSchema().getColumnDataType(0), DataSchema.ColumnDataType.INT);
+      Assert.assertEquals(block.getDataSchema().getColumnDataType(1), DataSchema.ColumnDataType.STRING);
+
+      List<String> expecteds =
+          Arrays.asList("[1, null]", "[2, null]", "[3, null]", "[4, null]", "[5, minney]", "[6, null]", "[7, scooby]");
+      int index = 0;
+
+      Iterator<Object[]> iterator = rows.iterator();
+      while (iterator.hasNext()) {
+        Object[] row = iterator.next();
+        System.out.println(Arrays.toString(row));
+        Assert.assertEquals(Arrays.toString(row), expecteds.get(index++));
+      }
+    } catch (IllegalStateException ise) {
+      Assert.assertTrue(true);

Review comment:
       This assert is an no-op. Do we expect exception here?

##########
File path: pinot-core/src/test/java/org/apache/pinot/queries/JsonIngestionFromAvroQueriesTest.java
##########
@@ -0,0 +1,305 @@
+/**
+ * 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.pinot.queries;
+
+import com.google.common.collect.Lists;
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.avro.file.DataFileWriter;
+import org.apache.avro.generic.GenericData;
+import org.apache.avro.generic.GenericDatumWriter;
+import org.apache.commons.io.FileUtils;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.core.common.Operator;
+import org.apache.pinot.core.operator.blocks.IntermediateResultsBlock;
+import org.apache.pinot.plugin.inputformat.avro.AvroRecordReader;
+import org.apache.pinot.segment.local.indexsegment.immutable.ImmutableSegmentLoader;
+import org.apache.pinot.segment.local.segment.creator.impl.SegmentIndexCreationDriverImpl;
+import org.apache.pinot.segment.local.segment.index.loader.IndexLoadingConfig;
+import org.apache.pinot.segment.spi.ImmutableSegment;
+import org.apache.pinot.segment.spi.IndexSegment;
+import org.apache.pinot.segment.spi.creator.SegmentGeneratorConfig;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.config.table.TableType;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.data.readers.RecordReader;
+import org.apache.pinot.spi.utils.Pair;
+import org.apache.pinot.spi.utils.ReadMode;
+import org.apache.pinot.spi.utils.builder.TableConfigBuilder;
+import org.testng.Assert;
+import org.testng.annotations.AfterClass;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+import static org.apache.avro.Schema.*;
+
+
+/**
+ * Test if ComplexType (RECORD, ARRAY, MAP, and UNION) field from an AVRO file can be ingested into a JSON column in
+ * a Pinot segment. This class tests. Ingestion from ENUM (symbol) and FIXED (binary) is not supported.
+ */
+public class JsonIngestionFromAvroQueriesTest extends BaseQueriesTest {
+  private static final File INDEX_DIR = new File(FileUtils.getTempDirectory(), "JsonIngestionFromAvroTest");
+  private static final File AVRO_DATA_FILE = new File(INDEX_DIR, "JsonIngestionFromAvroTest.avro");
+  private static final String RAW_TABLE_NAME = "testTable";
+  private static final String SEGMENT_NAME = "testSegment";
+
+  private static final String INT_COLUMN = "intColumn";
+  private static final String JSON_COLUMN = "jsonColumn";
+  private static final String STRING_COLUMN = "stringColumn";
+  private static final Schema SCHEMA =
+      new Schema.SchemaBuilder().addSingleValueDimension(INT_COLUMN, FieldSpec.DataType.INT)
+          .addSingleValueDimension(JSON_COLUMN, FieldSpec.DataType.JSON)
+          .addSingleValueDimension(STRING_COLUMN, FieldSpec.DataType.STRING).build();
+  private static final TableConfig TABLE_CONFIG =
+      new TableConfigBuilder(TableType.OFFLINE).setTableName(RAW_TABLE_NAME).build();
+
+  private IndexSegment _indexSegment;
+  private List<IndexSegment> _indexSegments;
+
+  @Override
+  protected String getFilter() {
+    return "";
+  }
+
+  @Override
+  protected IndexSegment getIndexSegment() {
+    return _indexSegment;
+  }
+
+  @Override
+  protected List<IndexSegment> getIndexSegments() {
+    return _indexSegments;
+  }
+
+  /** @return {@link GenericRow} representing a row in Pinot table. */
+  private static GenericRow createTableRecord(int intValue, String stringValue, Object jsonValue) {
+    GenericRow record = new GenericRow();
+    record.putValue(INT_COLUMN, intValue);
+    record.putValue(STRING_COLUMN, stringValue);
+    record.putValue(JSON_COLUMN, jsonValue);
+
+    return record;
+  }
+
+  private static Map<String, String> createMapField(Pair<String, String>[] pairs) {
+    Map<String, String> map = new HashMap<>();
+    for (Pair<String, String> pair : pairs) {
+      map.put(pair.getFirst(), pair.getSecond());
+    }
+    return map;
+  }
+
+  private static org.apache.avro.Schema createRecordSchema() {
+    List<org.apache.avro.Schema.Field> fields = new ArrayList<>();
+    fields.add(new org.apache.avro.Schema.Field("id", create(Type.INT)));
+    fields.add(new org.apache.avro.Schema.Field("name", create(Type.STRING)));
+    return createRecord("record", "doc", JsonIngestionFromAvroQueriesTest.class.getCanonicalName(), false, fields);
+  }
+
+  private static GenericData.Record createRecordField(String k1, int v1, String k2, String v2) {
+    GenericData.Record record = new GenericData.Record(createRecordSchema());
+    record.put(k1, v1);
+    record.put(k2, v2);
+    return record;
+  }
+
+  private static void createInputFile()
+      throws IOException {
+    INDEX_DIR.mkdir();
+    org.apache.avro.Schema avroSchema = org.apache.avro.Schema.createRecord("eventsRecord", null, null, false);
+    List<Field> fields = Arrays
+        .asList(new Field(INT_COLUMN, createUnion(Lists.newArrayList(create(Type.INT), create(Type.NULL))), null, null),
+            new Field(STRING_COLUMN, createUnion(Lists.newArrayList(create(Type.STRING), create(Type.NULL))), null,
+                null), new Field(JSON_COLUMN,
+                createUnion(createArray(create(Type.STRING)), createMap(create(Type.STRING)), createRecordSchema(),
+                    create(Type.STRING), create(Type.NULL))));
+    avroSchema.setFields(fields);
+    List<GenericRow> inputRecords = new ArrayList<>();
+    // Insert ARRAY
+    inputRecords.add(createTableRecord(1, "daffy duck", Arrays.asList("this", "is", "a", "test")));
+
+    // Insert MAP
+    inputRecords
+        .add(createTableRecord(2, "mickey mouse", createMapField(new Pair[]{new Pair("a", "1"), new Pair("b", "2")})));
+    inputRecords
+        .add(createTableRecord(3, "donald duck", createMapField(new Pair[]{new Pair("a", "1"), new Pair("b", "2")})));
+    inputRecords.add(
+        createTableRecord(4, "scrooge mcduck", createMapField(new Pair[]{new Pair("a", "1"), new Pair("b", "2")})));
+
+    // insert RECORD
+    inputRecords.add(createTableRecord(5, "minney mouse", createRecordField("id", 1, "name", "minney")));
+
+    // Insert simple Java String (gets converted into JSON value)
+    inputRecords.add(createTableRecord(6, "pluto", "test"));
+
+    // Insert JSON string (gets converted into JSON document)
+    inputRecords.add(createTableRecord(7, "scooby doo", "{\"name\":\"scooby\",\"id\":7}"));
+
+    try (DataFileWriter<GenericData.Record> fileWriter = new DataFileWriter<>(new GenericDatumWriter<>(avroSchema))) {
+      fileWriter.create(avroSchema, AVRO_DATA_FILE);
+      for (GenericRow inputRecord : inputRecords) {
+        GenericData.Record record = new GenericData.Record(avroSchema);
+        record.put(INT_COLUMN, inputRecord.getValue(INT_COLUMN));
+        record.put(STRING_COLUMN, inputRecord.getValue(STRING_COLUMN));
+        record.put(JSON_COLUMN, inputRecord.getValue(JSON_COLUMN));
+        fileWriter.append(record);
+      }
+    }
+  }
+
+  private static RecordReader createRecordReader()
+      throws IOException {
+    Set<String> set = new HashSet<>();
+    set.add(INT_COLUMN);
+    set.add(STRING_COLUMN);
+    set.add(JSON_COLUMN);
+    AvroRecordReader avroRecordReader = new AvroRecordReader();
+    avroRecordReader.init(AVRO_DATA_FILE, set, null);
+    return avroRecordReader;
+  }
+
+  /** Create an AVRO file and then ingest it into Pinot while creating a JsonIndex. */
+  @BeforeClass
+  public void setUp()
+      throws Exception {
+    FileUtils.deleteDirectory(INDEX_DIR);
+    createInputFile();
+
+    List<String> jsonIndexColumns = new ArrayList<>();
+    jsonIndexColumns.add("jsonColumn");
+    TABLE_CONFIG.getIndexingConfig().setJsonIndexColumns(jsonIndexColumns);
+    SegmentGeneratorConfig segmentGeneratorConfig = new SegmentGeneratorConfig(TABLE_CONFIG, SCHEMA);
+    segmentGeneratorConfig.setTableName(RAW_TABLE_NAME);
+    segmentGeneratorConfig.setSegmentName(SEGMENT_NAME);
+    segmentGeneratorConfig.setOutDir(INDEX_DIR.getPath());
+    segmentGeneratorConfig.setInputFilePath(AVRO_DATA_FILE.getPath());
+
+    SegmentIndexCreationDriverImpl driver = new SegmentIndexCreationDriverImpl();
+    driver.init(segmentGeneratorConfig, createRecordReader());
+    driver.build();
+
+    IndexLoadingConfig indexLoadingConfig = new IndexLoadingConfig();
+    indexLoadingConfig.setTableConfig(TABLE_CONFIG);
+    indexLoadingConfig.setJsonIndexColumns(new HashSet<String>(jsonIndexColumns));
+    indexLoadingConfig.setReadMode(ReadMode.mmap);
+
+    ImmutableSegment immutableSegment =
+        ImmutableSegmentLoader.load(new File(INDEX_DIR, SEGMENT_NAME), indexLoadingConfig);
+    _indexSegment = immutableSegment;
+    _indexSegments = Arrays.asList(immutableSegment, immutableSegment);
+  }
+
+  /** Verify that we can query the JSON column that ingested ComplexType data from an AVRO file (see setUp). */
+  @Test
+  public void testSimpleSelectOnJsonColumn() {
+    try {
+      Operator operator = getOperatorForSqlQuery("select intColumn, stringColumn, jsonColumn FROM testTable limit 100");
+      IntermediateResultsBlock block = (IntermediateResultsBlock) operator.nextBlock();
+      Collection<Object[]> rows = block.getSelectionResult();
+      Assert.assertEquals(block.getDataSchema().getColumnDataType(0), DataSchema.ColumnDataType.INT);
+      Assert.assertEquals(block.getDataSchema().getColumnDataType(1), DataSchema.ColumnDataType.STRING);
+      Assert.assertEquals(block.getDataSchema().getColumnDataType(2), DataSchema.ColumnDataType.JSON);
+
+      List<String> expecteds = Arrays
+          .asList("[1, daffy duck, [\"this\",\"is\",\"a\",\"test\"]]", "[2, mickey mouse, {\"a\":\"1\",\"b\":\"2\"}]",
+              "[3, donald duck, {\"a\":\"1\",\"b\":\"2\"}]", "[4, scrooge mcduck, {\"a\":\"1\",\"b\":\"2\"}]",
+              "[5, minney mouse, {\"name\":\"minney\",\"id\":1}]", "[6, pluto, \"test\"]",
+              "[7, scooby doo, {\"name\":\"scooby\",\"id\":7}]");
+      int index = 0;
+
+      Iterator<Object[]> iterator = rows.iterator();
+      while (iterator.hasNext()) {
+        Object[] row = iterator.next();
+        System.out.println(Arrays.toString(row));
+        Assert.assertEquals(Arrays.toString(row), expecteds.get(index++));
+      }
+    } catch (IllegalStateException ise) {
+      Assert.assertTrue(true);
+    }
+  }
+
+  /** Verify simple path expression query on ingested Avro file. */
+  @Test
+  public void testJsonPathSelectOnJsonColumn() {
+    try {
+      Operator operator = getOperatorForSqlQuery("select intColumn, json_extract_scalar(jsonColumn, '$.name', "
+          + "'STRING', 'null') FROM testTable");
+      IntermediateResultsBlock block = (IntermediateResultsBlock) operator.nextBlock();
+      Collection<Object[]> rows = block.getSelectionResult();
+      Assert.assertEquals(block.getDataSchema().getColumnDataType(0), DataSchema.ColumnDataType.INT);
+      Assert.assertEquals(block.getDataSchema().getColumnDataType(1), DataSchema.ColumnDataType.STRING);
+
+      List<String> expecteds =
+          Arrays.asList("[1, null]", "[2, null]", "[3, null]", "[4, null]", "[5, minney]", "[6, null]", "[7, scooby]");
+      int index = 0;
+
+      Iterator<Object[]> iterator = rows.iterator();
+      while (iterator.hasNext()) {
+        Object[] row = iterator.next();
+        System.out.println(Arrays.toString(row));

Review comment:
       Avoid console output

##########
File path: pinot-plugins/pinot-input-format/pinot-avro-base/src/main/java/org/apache/pinot/plugin/inputformat/avro/AvroIngestionSchemaValidator.java
##########
@@ -124,13 +124,16 @@ private void validateSchemas() {
               "The Pinot column: %s is 'single-value' column but the column: %s from input %s is 'multi-value' column.",
               columnName, avroColumnName, getInputSchemaType()));
         }
-        FieldSpec.DataType dataTypeForSVColumn = AvroUtils.extractFieldDataType(avroColumnField);
-        // check data type mismatch
-        if (fieldSpec.getDataType() != dataTypeForSVColumn) {
-          _dataTypeMismatch.addMismatchReason(String
-              .format("The Pinot column: (%s: %s) doesn't match with the column (%s: %s) in input %s schema.",
-                  columnName, fieldSpec.getDataType().name(), avroColumnName, avroColumnType.name(),
-                  getInputSchemaType()));
+
+        if (fieldSpec.getDataType() != FieldSpec.DataType.JSON) {

Review comment:
       We should skip the validation completely for JSON pinot type after validating the avro column field existence

##########
File path: pinot-core/src/test/java/org/apache/pinot/queries/JsonIngestionFromAvroQueriesTest.java
##########
@@ -0,0 +1,305 @@
+/**
+ * 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.pinot.queries;
+
+import com.google.common.collect.Lists;
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.avro.file.DataFileWriter;
+import org.apache.avro.generic.GenericData;
+import org.apache.avro.generic.GenericDatumWriter;
+import org.apache.commons.io.FileUtils;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.core.common.Operator;
+import org.apache.pinot.core.operator.blocks.IntermediateResultsBlock;
+import org.apache.pinot.plugin.inputformat.avro.AvroRecordReader;
+import org.apache.pinot.segment.local.indexsegment.immutable.ImmutableSegmentLoader;
+import org.apache.pinot.segment.local.segment.creator.impl.SegmentIndexCreationDriverImpl;
+import org.apache.pinot.segment.local.segment.index.loader.IndexLoadingConfig;
+import org.apache.pinot.segment.spi.ImmutableSegment;
+import org.apache.pinot.segment.spi.IndexSegment;
+import org.apache.pinot.segment.spi.creator.SegmentGeneratorConfig;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.config.table.TableType;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.data.readers.RecordReader;
+import org.apache.pinot.spi.utils.Pair;
+import org.apache.pinot.spi.utils.ReadMode;
+import org.apache.pinot.spi.utils.builder.TableConfigBuilder;
+import org.testng.Assert;
+import org.testng.annotations.AfterClass;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+import static org.apache.avro.Schema.*;
+
+
+/**
+ * Test if ComplexType (RECORD, ARRAY, MAP, and UNION) field from an AVRO file can be ingested into a JSON column in
+ * a Pinot segment. This class tests. Ingestion from ENUM (symbol) and FIXED (binary) is not supported.
+ */
+public class JsonIngestionFromAvroQueriesTest extends BaseQueriesTest {
+  private static final File INDEX_DIR = new File(FileUtils.getTempDirectory(), "JsonIngestionFromAvroTest");
+  private static final File AVRO_DATA_FILE = new File(INDEX_DIR, "JsonIngestionFromAvroTest.avro");
+  private static final String RAW_TABLE_NAME = "testTable";
+  private static final String SEGMENT_NAME = "testSegment";
+
+  private static final String INT_COLUMN = "intColumn";
+  private static final String JSON_COLUMN = "jsonColumn";
+  private static final String STRING_COLUMN = "stringColumn";
+  private static final Schema SCHEMA =
+      new Schema.SchemaBuilder().addSingleValueDimension(INT_COLUMN, FieldSpec.DataType.INT)
+          .addSingleValueDimension(JSON_COLUMN, FieldSpec.DataType.JSON)
+          .addSingleValueDimension(STRING_COLUMN, FieldSpec.DataType.STRING).build();
+  private static final TableConfig TABLE_CONFIG =
+      new TableConfigBuilder(TableType.OFFLINE).setTableName(RAW_TABLE_NAME).build();
+
+  private IndexSegment _indexSegment;
+  private List<IndexSegment> _indexSegments;
+
+  @Override
+  protected String getFilter() {
+    return "";
+  }
+
+  @Override
+  protected IndexSegment getIndexSegment() {
+    return _indexSegment;
+  }
+
+  @Override
+  protected List<IndexSegment> getIndexSegments() {
+    return _indexSegments;
+  }
+
+  /** @return {@link GenericRow} representing a row in Pinot table. */
+  private static GenericRow createTableRecord(int intValue, String stringValue, Object jsonValue) {
+    GenericRow record = new GenericRow();
+    record.putValue(INT_COLUMN, intValue);
+    record.putValue(STRING_COLUMN, stringValue);
+    record.putValue(JSON_COLUMN, jsonValue);
+
+    return record;
+  }
+
+  private static Map<String, String> createMapField(Pair<String, String>[] pairs) {
+    Map<String, String> map = new HashMap<>();
+    for (Pair<String, String> pair : pairs) {
+      map.put(pair.getFirst(), pair.getSecond());
+    }
+    return map;
+  }
+
+  private static org.apache.avro.Schema createRecordSchema() {
+    List<org.apache.avro.Schema.Field> fields = new ArrayList<>();

Review comment:
       This can be simplified
   ```suggestion
       List<Field> fields = new ArrayList<>();
   ```




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

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [pinot] amrishlal commented on pull request #8426: Allow ingesting Avro complex types into JSON column.

Posted by GitBox <gi...@apache.org>.
amrishlal commented on pull request #8426:
URL: https://github.com/apache/pinot/pull/8426#issuecomment-1082478220


   @siddharthteotia please review.


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

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [pinot] codecov-commenter edited a comment on pull request #8426: Allow ingesting Avro complex types into JSON column.

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #8426:
URL: https://github.com/apache/pinot/pull/8426#issuecomment-1081210576


   # [Codecov](https://codecov.io/gh/apache/pinot/pull/8426?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#8426](https://codecov.io/gh/apache/pinot/pull/8426?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (48b300a) into [master](https://codecov.io/gh/apache/pinot/commit/dfc7ea8a91f0e5d1f99559f6b094d1ccc1dff5a4?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (dfc7ea8) will **decrease** coverage by `0.05%`.
   > The diff coverage is `36.36%`.
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #8426      +/-   ##
   ============================================
   - Coverage     69.66%   69.60%   -0.06%     
   - Complexity     4238     4280      +42     
   ============================================
     Files          1629     1660      +31     
     Lines         85140    87039    +1899     
     Branches      12815    13140     +325     
   ============================================
   + Hits          59310    60586    +1276     
   - Misses        21698    22226     +528     
   - Partials       4132     4227      +95     
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | integration1 | `?` | |
   | integration2 | `27.37% <0.00%> (?)` | |
   | unittests1 | `66.99% <36.36%> (+0.07%)` | :arrow_up: |
   | unittests2 | `14.19% <0.00%> (+0.08%)` | :arrow_up: |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/pinot/pull/8426?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...inputformat/avro/AvroIngestionSchemaValidator.java](https://codecov.io/gh/apache/pinot/pull/8426/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtcGx1Z2lucy9waW5vdC1pbnB1dC1mb3JtYXQvcGlub3QtYXZyby1iYXNlL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9wbHVnaW4vaW5wdXRmb3JtYXQvYXZyby9BdnJvSW5nZXN0aW9uU2NoZW1hVmFsaWRhdG9yLmphdmE=) | `0.00% <0.00%> (ø)` | |
   | [.../pinot/plugin/inputformat/avro/AvroSchemaUtil.java](https://codecov.io/gh/apache/pinot/pull/8426/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtcGx1Z2lucy9waW5vdC1pbnB1dC1mb3JtYXQvcGlub3QtYXZyby1iYXNlL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9wbHVnaW4vaW5wdXRmb3JtYXQvYXZyby9BdnJvU2NoZW1hVXRpbC5qYXZh) | `18.18% <0.00%> (-0.57%)` | :arrow_down: |
   | [...a/org/apache/pinot/common/utils/PinotDataType.java](https://codecov.io/gh/apache/pinot/pull/8426/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vdXRpbHMvUGlub3REYXRhVHlwZS5qYXZh) | `81.06% <33.33%> (-0.37%)` | :arrow_down: |
   | [...t/local/recordtransformer/DataTypeTransformer.java](https://codecov.io/gh/apache/pinot/pull/8426/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC9yZWNvcmR0cmFuc2Zvcm1lci9EYXRhVHlwZVRyYW5zZm9ybWVyLmphdmE=) | `89.61% <100.00%> (+2.76%)` | :arrow_up: |
   | [...pinot/minion/exception/TaskCancelledException.java](https://codecov.io/gh/apache/pinot/pull/8426/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtbWluaW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9taW5pb24vZXhjZXB0aW9uL1Rhc2tDYW5jZWxsZWRFeGNlcHRpb24uamF2YQ==) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...nverttorawindex/ConvertToRawIndexTaskExecutor.java](https://codecov.io/gh/apache/pinot/pull/8426/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtcGx1Z2lucy9waW5vdC1taW5pb24tdGFza3MvcGlub3QtbWluaW9uLWJ1aWx0aW4tdGFza3Mvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3Bpbm90L3BsdWdpbi9taW5pb24vdGFza3MvY29udmVydHRvcmF3aW5kZXgvQ29udmVydFRvUmF3SW5kZXhUYXNrRXhlY3V0b3IuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...e/pinot/common/minion/MergeRollupTaskMetadata.java](https://codecov.io/gh/apache/pinot/pull/8426/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vbWluaW9uL01lcmdlUm9sbHVwVGFza01ldGFkYXRhLmphdmE=) | `0.00% <0.00%> (-94.74%)` | :arrow_down: |
   | [...plugin/segmentuploader/SegmentUploaderDefault.java](https://codecov.io/gh/apache/pinot/pull/8426/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtcGx1Z2lucy9waW5vdC1zZWdtZW50LXVwbG9hZGVyL3Bpbm90LXNlZ21lbnQtdXBsb2FkZXItZGVmYXVsdC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvcGx1Z2luL3NlZ21lbnR1cGxvYWRlci9TZWdtZW50VXBsb2FkZXJEZWZhdWx0LmphdmE=) | `0.00% <0.00%> (-87.10%)` | :arrow_down: |
   | [.../transform/function/MapValueTransformFunction.java](https://codecov.io/gh/apache/pinot/pull/8426/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9vcGVyYXRvci90cmFuc2Zvcm0vZnVuY3Rpb24vTWFwVmFsdWVUcmFuc2Zvcm1GdW5jdGlvbi5qYXZh) | `0.00% <0.00%> (-85.30%)` | :arrow_down: |
   | [...ot/common/messages/RoutingTableRebuildMessage.java](https://codecov.io/gh/apache/pinot/pull/8426/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vbWVzc2FnZXMvUm91dGluZ1RhYmxlUmVidWlsZE1lc3NhZ2UuamF2YQ==) | `0.00% <0.00%> (-81.82%)` | :arrow_down: |
   | ... and [358 more](https://codecov.io/gh/apache/pinot/pull/8426/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/pinot/pull/8426?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/pinot/pull/8426?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [dfc7ea8...48b300a](https://codecov.io/gh/apache/pinot/pull/8426?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   


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

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [pinot] codecov-commenter edited a comment on pull request #8426: Allow ingesting Avro complex types into JSON column.

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #8426:
URL: https://github.com/apache/pinot/pull/8426#issuecomment-1081210576


   # [Codecov](https://codecov.io/gh/apache/pinot/pull/8426?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#8426](https://codecov.io/gh/apache/pinot/pull/8426?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (3c9ccb8) into [master](https://codecov.io/gh/apache/pinot/commit/dfc7ea8a91f0e5d1f99559f6b094d1ccc1dff5a4?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (dfc7ea8) will **decrease** coverage by `55.55%`.
   > The diff coverage is `0.00%`.
   
   ```diff
   @@              Coverage Diff              @@
   ##             master    #8426       +/-   ##
   =============================================
   - Coverage     69.66%   14.10%   -55.56%     
   + Complexity     4238       84     -4154     
   =============================================
     Files          1629     1611       -18     
     Lines         85140    84817      -323     
     Branches      12815    12875       +60     
   =============================================
   - Hits          59310    11965    -47345     
   - Misses        21698    71955    +50257     
   + Partials       4132      897     -3235     
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | integration1 | `?` | |
   | unittests1 | `?` | |
   | unittests2 | `14.10% <0.00%> (-0.01%)` | :arrow_down: |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/pinot/pull/8426?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...a/org/apache/pinot/common/utils/PinotDataType.java](https://codecov.io/gh/apache/pinot/pull/8426/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vdXRpbHMvUGlub3REYXRhVHlwZS5qYXZh) | `0.00% <0.00%> (-81.44%)` | :arrow_down: |
   | [...inputformat/avro/AvroIngestionSchemaValidator.java](https://codecov.io/gh/apache/pinot/pull/8426/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtcGx1Z2lucy9waW5vdC1pbnB1dC1mb3JtYXQvcGlub3QtYXZyby1iYXNlL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9wbHVnaW4vaW5wdXRmb3JtYXQvYXZyby9BdnJvSW5nZXN0aW9uU2NoZW1hVmFsaWRhdG9yLmphdmE=) | `0.00% <0.00%> (ø)` | |
   | [.../pinot/plugin/inputformat/avro/AvroSchemaUtil.java](https://codecov.io/gh/apache/pinot/pull/8426/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtcGx1Z2lucy9waW5vdC1pbnB1dC1mb3JtYXQvcGlub3QtYXZyby1iYXNlL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9wbHVnaW4vaW5wdXRmb3JtYXQvYXZyby9BdnJvU2NoZW1hVXRpbC5qYXZh) | `0.00% <0.00%> (-18.75%)` | :arrow_down: |
   | [...t/local/recordtransformer/DataTypeTransformer.java](https://codecov.io/gh/apache/pinot/pull/8426/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC9yZWNvcmR0cmFuc2Zvcm1lci9EYXRhVHlwZVRyYW5zZm9ybWVyLmphdmE=) | `0.00% <0.00%> (-86.85%)` | :arrow_down: |
   | [...ain/java/org/apache/pinot/core/data/table/Key.java](https://codecov.io/gh/apache/pinot/pull/8426/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9kYXRhL3RhYmxlL0tleS5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [.../java/org/apache/pinot/spi/utils/BooleanUtils.java](https://codecov.io/gh/apache/pinot/pull/8426/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3Qtc3BpL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9zcGkvdXRpbHMvQm9vbGVhblV0aWxzLmphdmE=) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [.../java/org/apache/pinot/core/data/table/Record.java](https://codecov.io/gh/apache/pinot/pull/8426/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9kYXRhL3RhYmxlL1JlY29yZC5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [.../java/org/apache/pinot/core/util/GroupByUtils.java](https://codecov.io/gh/apache/pinot/pull/8426/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS91dGlsL0dyb3VwQnlVdGlscy5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...ava/org/apache/pinot/spi/config/table/FSTType.java](https://codecov.io/gh/apache/pinot/pull/8426/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3Qtc3BpL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9zcGkvY29uZmlnL3RhYmxlL0ZTVFR5cGUuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...ava/org/apache/pinot/spi/data/MetricFieldSpec.java](https://codecov.io/gh/apache/pinot/pull/8426/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3Qtc3BpL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9zcGkvZGF0YS9NZXRyaWNGaWVsZFNwZWMuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | ... and [1341 more](https://codecov.io/gh/apache/pinot/pull/8426/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/pinot/pull/8426?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/pinot/pull/8426?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [dfc7ea8...3c9ccb8](https://codecov.io/gh/apache/pinot/pull/8426?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   


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

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [pinot] amrishlal commented on a change in pull request #8426: Allow ingesting Avro complex types into JSON column.

Posted by GitBox <gi...@apache.org>.
amrishlal commented on a change in pull request #8426:
URL: https://github.com/apache/pinot/pull/8426#discussion_r836927599



##########
File path: pinot-common/src/main/java/org/apache/pinot/common/utils/PinotDataType.java
##########
@@ -829,18 +829,23 @@ public String toString(Object value) {
   public String toJson(Object value) {
     if (value instanceof String) {
       try {
+        // Try to parse the string as JSON first
         return JsonUtils.stringToJsonNode((String) value).toString();
+      } catch (com.fasterxml.jackson.core.JsonParseException jpe) {

Review comment:
       Done.

##########
File path: pinot-core/src/test/java/org/apache/pinot/queries/JsonIngestionFromAvroQueriesTest.java
##########
@@ -0,0 +1,305 @@
+/**
+ * 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.pinot.queries;
+
+import com.google.common.collect.Lists;
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.avro.file.DataFileWriter;
+import org.apache.avro.generic.GenericData;
+import org.apache.avro.generic.GenericDatumWriter;
+import org.apache.commons.io.FileUtils;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.core.common.Operator;
+import org.apache.pinot.core.operator.blocks.IntermediateResultsBlock;
+import org.apache.pinot.plugin.inputformat.avro.AvroRecordReader;
+import org.apache.pinot.segment.local.indexsegment.immutable.ImmutableSegmentLoader;
+import org.apache.pinot.segment.local.segment.creator.impl.SegmentIndexCreationDriverImpl;
+import org.apache.pinot.segment.local.segment.index.loader.IndexLoadingConfig;
+import org.apache.pinot.segment.spi.ImmutableSegment;
+import org.apache.pinot.segment.spi.IndexSegment;
+import org.apache.pinot.segment.spi.creator.SegmentGeneratorConfig;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.config.table.TableType;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.data.readers.RecordReader;
+import org.apache.pinot.spi.utils.Pair;
+import org.apache.pinot.spi.utils.ReadMode;
+import org.apache.pinot.spi.utils.builder.TableConfigBuilder;
+import org.testng.Assert;
+import org.testng.annotations.AfterClass;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+import static org.apache.avro.Schema.*;
+
+
+/**
+ * Test if ComplexType (RECORD, ARRAY, MAP, and UNION) field from an AVRO file can be ingested into a JSON column in
+ * a Pinot segment. This class tests. Ingestion from ENUM (symbol) and FIXED (binary) is not supported.
+ */
+public class JsonIngestionFromAvroQueriesTest extends BaseQueriesTest {
+  private static final File INDEX_DIR = new File(FileUtils.getTempDirectory(), "JsonIngestionFromAvroTest");
+  private static final File AVRO_DATA_FILE = new File(INDEX_DIR, "JsonIngestionFromAvroTest.avro");
+  private static final String RAW_TABLE_NAME = "testTable";
+  private static final String SEGMENT_NAME = "testSegment";
+
+  private static final String INT_COLUMN = "intColumn";
+  private static final String JSON_COLUMN = "jsonColumn";
+  private static final String STRING_COLUMN = "stringColumn";
+  private static final Schema SCHEMA =
+      new Schema.SchemaBuilder().addSingleValueDimension(INT_COLUMN, FieldSpec.DataType.INT)
+          .addSingleValueDimension(JSON_COLUMN, FieldSpec.DataType.JSON)
+          .addSingleValueDimension(STRING_COLUMN, FieldSpec.DataType.STRING).build();
+  private static final TableConfig TABLE_CONFIG =
+      new TableConfigBuilder(TableType.OFFLINE).setTableName(RAW_TABLE_NAME).build();
+
+  private IndexSegment _indexSegment;
+  private List<IndexSegment> _indexSegments;
+
+  @Override
+  protected String getFilter() {
+    return "";
+  }
+
+  @Override
+  protected IndexSegment getIndexSegment() {
+    return _indexSegment;
+  }
+
+  @Override
+  protected List<IndexSegment> getIndexSegments() {
+    return _indexSegments;
+  }
+
+  /** @return {@link GenericRow} representing a row in Pinot table. */
+  private static GenericRow createTableRecord(int intValue, String stringValue, Object jsonValue) {
+    GenericRow record = new GenericRow();
+    record.putValue(INT_COLUMN, intValue);
+    record.putValue(STRING_COLUMN, stringValue);
+    record.putValue(JSON_COLUMN, jsonValue);
+
+    return record;
+  }
+
+  private static Map<String, String> createMapField(Pair<String, String>[] pairs) {
+    Map<String, String> map = new HashMap<>();
+    for (Pair<String, String> pair : pairs) {
+      map.put(pair.getFirst(), pair.getSecond());
+    }
+    return map;
+  }
+
+  private static org.apache.avro.Schema createRecordSchema() {
+    List<org.apache.avro.Schema.Field> fields = new ArrayList<>();

Review comment:
       Done.

##########
File path: pinot-core/src/test/java/org/apache/pinot/queries/JsonIngestionFromAvroQueriesTest.java
##########
@@ -0,0 +1,305 @@
+/**
+ * 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.pinot.queries;
+
+import com.google.common.collect.Lists;
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.avro.file.DataFileWriter;
+import org.apache.avro.generic.GenericData;
+import org.apache.avro.generic.GenericDatumWriter;
+import org.apache.commons.io.FileUtils;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.core.common.Operator;
+import org.apache.pinot.core.operator.blocks.IntermediateResultsBlock;
+import org.apache.pinot.plugin.inputformat.avro.AvroRecordReader;
+import org.apache.pinot.segment.local.indexsegment.immutable.ImmutableSegmentLoader;
+import org.apache.pinot.segment.local.segment.creator.impl.SegmentIndexCreationDriverImpl;
+import org.apache.pinot.segment.local.segment.index.loader.IndexLoadingConfig;
+import org.apache.pinot.segment.spi.ImmutableSegment;
+import org.apache.pinot.segment.spi.IndexSegment;
+import org.apache.pinot.segment.spi.creator.SegmentGeneratorConfig;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.config.table.TableType;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.data.readers.RecordReader;
+import org.apache.pinot.spi.utils.Pair;
+import org.apache.pinot.spi.utils.ReadMode;
+import org.apache.pinot.spi.utils.builder.TableConfigBuilder;
+import org.testng.Assert;
+import org.testng.annotations.AfterClass;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+import static org.apache.avro.Schema.*;

Review comment:
       Done.

##########
File path: pinot-core/src/test/java/org/apache/pinot/queries/JsonIngestionFromAvroQueriesTest.java
##########
@@ -0,0 +1,305 @@
+/**
+ * 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.pinot.queries;
+
+import com.google.common.collect.Lists;
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.avro.file.DataFileWriter;
+import org.apache.avro.generic.GenericData;
+import org.apache.avro.generic.GenericDatumWriter;
+import org.apache.commons.io.FileUtils;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.core.common.Operator;
+import org.apache.pinot.core.operator.blocks.IntermediateResultsBlock;
+import org.apache.pinot.plugin.inputformat.avro.AvroRecordReader;
+import org.apache.pinot.segment.local.indexsegment.immutable.ImmutableSegmentLoader;
+import org.apache.pinot.segment.local.segment.creator.impl.SegmentIndexCreationDriverImpl;
+import org.apache.pinot.segment.local.segment.index.loader.IndexLoadingConfig;
+import org.apache.pinot.segment.spi.ImmutableSegment;
+import org.apache.pinot.segment.spi.IndexSegment;
+import org.apache.pinot.segment.spi.creator.SegmentGeneratorConfig;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.config.table.TableType;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.data.readers.RecordReader;
+import org.apache.pinot.spi.utils.Pair;
+import org.apache.pinot.spi.utils.ReadMode;
+import org.apache.pinot.spi.utils.builder.TableConfigBuilder;
+import org.testng.Assert;
+import org.testng.annotations.AfterClass;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+import static org.apache.avro.Schema.*;
+
+
+/**
+ * Test if ComplexType (RECORD, ARRAY, MAP, and UNION) field from an AVRO file can be ingested into a JSON column in
+ * a Pinot segment. This class tests. Ingestion from ENUM (symbol) and FIXED (binary) is not supported.
+ */
+public class JsonIngestionFromAvroQueriesTest extends BaseQueriesTest {
+  private static final File INDEX_DIR = new File(FileUtils.getTempDirectory(), "JsonIngestionFromAvroTest");
+  private static final File AVRO_DATA_FILE = new File(INDEX_DIR, "JsonIngestionFromAvroTest.avro");
+  private static final String RAW_TABLE_NAME = "testTable";
+  private static final String SEGMENT_NAME = "testSegment";
+
+  private static final String INT_COLUMN = "intColumn";
+  private static final String JSON_COLUMN = "jsonColumn";
+  private static final String STRING_COLUMN = "stringColumn";
+  private static final Schema SCHEMA =
+      new Schema.SchemaBuilder().addSingleValueDimension(INT_COLUMN, FieldSpec.DataType.INT)
+          .addSingleValueDimension(JSON_COLUMN, FieldSpec.DataType.JSON)
+          .addSingleValueDimension(STRING_COLUMN, FieldSpec.DataType.STRING).build();
+  private static final TableConfig TABLE_CONFIG =
+      new TableConfigBuilder(TableType.OFFLINE).setTableName(RAW_TABLE_NAME).build();
+
+  private IndexSegment _indexSegment;
+  private List<IndexSegment> _indexSegments;
+
+  @Override
+  protected String getFilter() {
+    return "";
+  }
+
+  @Override
+  protected IndexSegment getIndexSegment() {
+    return _indexSegment;
+  }
+
+  @Override
+  protected List<IndexSegment> getIndexSegments() {
+    return _indexSegments;
+  }
+
+  /** @return {@link GenericRow} representing a row in Pinot table. */
+  private static GenericRow createTableRecord(int intValue, String stringValue, Object jsonValue) {
+    GenericRow record = new GenericRow();
+    record.putValue(INT_COLUMN, intValue);
+    record.putValue(STRING_COLUMN, stringValue);
+    record.putValue(JSON_COLUMN, jsonValue);
+
+    return record;
+  }
+
+  private static Map<String, String> createMapField(Pair<String, String>[] pairs) {
+    Map<String, String> map = new HashMap<>();
+    for (Pair<String, String> pair : pairs) {
+      map.put(pair.getFirst(), pair.getSecond());
+    }
+    return map;
+  }
+
+  private static org.apache.avro.Schema createRecordSchema() {
+    List<org.apache.avro.Schema.Field> fields = new ArrayList<>();
+    fields.add(new org.apache.avro.Schema.Field("id", create(Type.INT)));
+    fields.add(new org.apache.avro.Schema.Field("name", create(Type.STRING)));
+    return createRecord("record", "doc", JsonIngestionFromAvroQueriesTest.class.getCanonicalName(), false, fields);
+  }
+
+  private static GenericData.Record createRecordField(String k1, int v1, String k2, String v2) {
+    GenericData.Record record = new GenericData.Record(createRecordSchema());
+    record.put(k1, v1);
+    record.put(k2, v2);
+    return record;
+  }
+
+  private static void createInputFile()
+      throws IOException {
+    INDEX_DIR.mkdir();
+    org.apache.avro.Schema avroSchema = org.apache.avro.Schema.createRecord("eventsRecord", null, null, false);
+    List<Field> fields = Arrays
+        .asList(new Field(INT_COLUMN, createUnion(Lists.newArrayList(create(Type.INT), create(Type.NULL))), null, null),
+            new Field(STRING_COLUMN, createUnion(Lists.newArrayList(create(Type.STRING), create(Type.NULL))), null,
+                null), new Field(JSON_COLUMN,
+                createUnion(createArray(create(Type.STRING)), createMap(create(Type.STRING)), createRecordSchema(),
+                    create(Type.STRING), create(Type.NULL))));
+    avroSchema.setFields(fields);
+    List<GenericRow> inputRecords = new ArrayList<>();
+    // Insert ARRAY
+    inputRecords.add(createTableRecord(1, "daffy duck", Arrays.asList("this", "is", "a", "test")));
+
+    // Insert MAP
+    inputRecords
+        .add(createTableRecord(2, "mickey mouse", createMapField(new Pair[]{new Pair("a", "1"), new Pair("b", "2")})));
+    inputRecords
+        .add(createTableRecord(3, "donald duck", createMapField(new Pair[]{new Pair("a", "1"), new Pair("b", "2")})));
+    inputRecords.add(
+        createTableRecord(4, "scrooge mcduck", createMapField(new Pair[]{new Pair("a", "1"), new Pair("b", "2")})));
+
+    // insert RECORD
+    inputRecords.add(createTableRecord(5, "minney mouse", createRecordField("id", 1, "name", "minney")));
+
+    // Insert simple Java String (gets converted into JSON value)
+    inputRecords.add(createTableRecord(6, "pluto", "test"));
+
+    // Insert JSON string (gets converted into JSON document)
+    inputRecords.add(createTableRecord(7, "scooby doo", "{\"name\":\"scooby\",\"id\":7}"));
+
+    try (DataFileWriter<GenericData.Record> fileWriter = new DataFileWriter<>(new GenericDatumWriter<>(avroSchema))) {
+      fileWriter.create(avroSchema, AVRO_DATA_FILE);
+      for (GenericRow inputRecord : inputRecords) {
+        GenericData.Record record = new GenericData.Record(avroSchema);
+        record.put(INT_COLUMN, inputRecord.getValue(INT_COLUMN));
+        record.put(STRING_COLUMN, inputRecord.getValue(STRING_COLUMN));
+        record.put(JSON_COLUMN, inputRecord.getValue(JSON_COLUMN));
+        fileWriter.append(record);
+      }
+    }
+  }
+
+  private static RecordReader createRecordReader()
+      throws IOException {
+    Set<String> set = new HashSet<>();
+    set.add(INT_COLUMN);
+    set.add(STRING_COLUMN);
+    set.add(JSON_COLUMN);
+    AvroRecordReader avroRecordReader = new AvroRecordReader();
+    avroRecordReader.init(AVRO_DATA_FILE, set, null);
+    return avroRecordReader;
+  }
+
+  /** Create an AVRO file and then ingest it into Pinot while creating a JsonIndex. */
+  @BeforeClass
+  public void setUp()
+      throws Exception {
+    FileUtils.deleteDirectory(INDEX_DIR);
+    createInputFile();
+
+    List<String> jsonIndexColumns = new ArrayList<>();
+    jsonIndexColumns.add("jsonColumn");
+    TABLE_CONFIG.getIndexingConfig().setJsonIndexColumns(jsonIndexColumns);
+    SegmentGeneratorConfig segmentGeneratorConfig = new SegmentGeneratorConfig(TABLE_CONFIG, SCHEMA);
+    segmentGeneratorConfig.setTableName(RAW_TABLE_NAME);
+    segmentGeneratorConfig.setSegmentName(SEGMENT_NAME);
+    segmentGeneratorConfig.setOutDir(INDEX_DIR.getPath());
+    segmentGeneratorConfig.setInputFilePath(AVRO_DATA_FILE.getPath());
+
+    SegmentIndexCreationDriverImpl driver = new SegmentIndexCreationDriverImpl();
+    driver.init(segmentGeneratorConfig, createRecordReader());
+    driver.build();
+
+    IndexLoadingConfig indexLoadingConfig = new IndexLoadingConfig();
+    indexLoadingConfig.setTableConfig(TABLE_CONFIG);
+    indexLoadingConfig.setJsonIndexColumns(new HashSet<String>(jsonIndexColumns));
+    indexLoadingConfig.setReadMode(ReadMode.mmap);
+
+    ImmutableSegment immutableSegment =
+        ImmutableSegmentLoader.load(new File(INDEX_DIR, SEGMENT_NAME), indexLoadingConfig);
+    _indexSegment = immutableSegment;
+    _indexSegments = Arrays.asList(immutableSegment, immutableSegment);
+  }
+
+  /** Verify that we can query the JSON column that ingested ComplexType data from an AVRO file (see setUp). */
+  @Test
+  public void testSimpleSelectOnJsonColumn() {
+    try {
+      Operator operator = getOperatorForSqlQuery("select intColumn, stringColumn, jsonColumn FROM testTable limit 100");
+      IntermediateResultsBlock block = (IntermediateResultsBlock) operator.nextBlock();
+      Collection<Object[]> rows = block.getSelectionResult();
+      Assert.assertEquals(block.getDataSchema().getColumnDataType(0), DataSchema.ColumnDataType.INT);
+      Assert.assertEquals(block.getDataSchema().getColumnDataType(1), DataSchema.ColumnDataType.STRING);
+      Assert.assertEquals(block.getDataSchema().getColumnDataType(2), DataSchema.ColumnDataType.JSON);
+
+      List<String> expecteds = Arrays
+          .asList("[1, daffy duck, [\"this\",\"is\",\"a\",\"test\"]]", "[2, mickey mouse, {\"a\":\"1\",\"b\":\"2\"}]",
+              "[3, donald duck, {\"a\":\"1\",\"b\":\"2\"}]", "[4, scrooge mcduck, {\"a\":\"1\",\"b\":\"2\"}]",
+              "[5, minney mouse, {\"name\":\"minney\",\"id\":1}]", "[6, pluto, \"test\"]",
+              "[7, scooby doo, {\"name\":\"scooby\",\"id\":7}]");
+      int index = 0;
+
+      Iterator<Object[]> iterator = rows.iterator();
+      while (iterator.hasNext()) {
+        Object[] row = iterator.next();
+        System.out.println(Arrays.toString(row));
+        Assert.assertEquals(Arrays.toString(row), expecteds.get(index++));
+      }
+    } catch (IllegalStateException ise) {
+      Assert.assertTrue(true);
+    }
+  }
+
+  /** Verify simple path expression query on ingested Avro file. */
+  @Test
+  public void testJsonPathSelectOnJsonColumn() {
+    try {
+      Operator operator = getOperatorForSqlQuery("select intColumn, json_extract_scalar(jsonColumn, '$.name', "
+          + "'STRING', 'null') FROM testTable");
+      IntermediateResultsBlock block = (IntermediateResultsBlock) operator.nextBlock();
+      Collection<Object[]> rows = block.getSelectionResult();
+      Assert.assertEquals(block.getDataSchema().getColumnDataType(0), DataSchema.ColumnDataType.INT);
+      Assert.assertEquals(block.getDataSchema().getColumnDataType(1), DataSchema.ColumnDataType.STRING);
+
+      List<String> expecteds =
+          Arrays.asList("[1, null]", "[2, null]", "[3, null]", "[4, null]", "[5, minney]", "[6, null]", "[7, scooby]");
+      int index = 0;
+
+      Iterator<Object[]> iterator = rows.iterator();
+      while (iterator.hasNext()) {
+        Object[] row = iterator.next();
+        System.out.println(Arrays.toString(row));

Review comment:
       Done.

##########
File path: pinot-core/src/test/java/org/apache/pinot/queries/JsonIngestionFromAvroQueriesTest.java
##########
@@ -0,0 +1,305 @@
+/**
+ * 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.pinot.queries;
+
+import com.google.common.collect.Lists;
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.avro.file.DataFileWriter;
+import org.apache.avro.generic.GenericData;
+import org.apache.avro.generic.GenericDatumWriter;
+import org.apache.commons.io.FileUtils;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.core.common.Operator;
+import org.apache.pinot.core.operator.blocks.IntermediateResultsBlock;
+import org.apache.pinot.plugin.inputformat.avro.AvroRecordReader;
+import org.apache.pinot.segment.local.indexsegment.immutable.ImmutableSegmentLoader;
+import org.apache.pinot.segment.local.segment.creator.impl.SegmentIndexCreationDriverImpl;
+import org.apache.pinot.segment.local.segment.index.loader.IndexLoadingConfig;
+import org.apache.pinot.segment.spi.ImmutableSegment;
+import org.apache.pinot.segment.spi.IndexSegment;
+import org.apache.pinot.segment.spi.creator.SegmentGeneratorConfig;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.config.table.TableType;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.data.readers.RecordReader;
+import org.apache.pinot.spi.utils.Pair;
+import org.apache.pinot.spi.utils.ReadMode;
+import org.apache.pinot.spi.utils.builder.TableConfigBuilder;
+import org.testng.Assert;
+import org.testng.annotations.AfterClass;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+import static org.apache.avro.Schema.*;
+
+
+/**
+ * Test if ComplexType (RECORD, ARRAY, MAP, and UNION) field from an AVRO file can be ingested into a JSON column in
+ * a Pinot segment. This class tests. Ingestion from ENUM (symbol) and FIXED (binary) is not supported.
+ */
+public class JsonIngestionFromAvroQueriesTest extends BaseQueriesTest {
+  private static final File INDEX_DIR = new File(FileUtils.getTempDirectory(), "JsonIngestionFromAvroTest");
+  private static final File AVRO_DATA_FILE = new File(INDEX_DIR, "JsonIngestionFromAvroTest.avro");
+  private static final String RAW_TABLE_NAME = "testTable";
+  private static final String SEGMENT_NAME = "testSegment";
+
+  private static final String INT_COLUMN = "intColumn";
+  private static final String JSON_COLUMN = "jsonColumn";
+  private static final String STRING_COLUMN = "stringColumn";
+  private static final Schema SCHEMA =
+      new Schema.SchemaBuilder().addSingleValueDimension(INT_COLUMN, FieldSpec.DataType.INT)
+          .addSingleValueDimension(JSON_COLUMN, FieldSpec.DataType.JSON)
+          .addSingleValueDimension(STRING_COLUMN, FieldSpec.DataType.STRING).build();
+  private static final TableConfig TABLE_CONFIG =
+      new TableConfigBuilder(TableType.OFFLINE).setTableName(RAW_TABLE_NAME).build();
+
+  private IndexSegment _indexSegment;
+  private List<IndexSegment> _indexSegments;
+
+  @Override
+  protected String getFilter() {
+    return "";
+  }
+
+  @Override
+  protected IndexSegment getIndexSegment() {
+    return _indexSegment;
+  }
+
+  @Override
+  protected List<IndexSegment> getIndexSegments() {
+    return _indexSegments;
+  }
+
+  /** @return {@link GenericRow} representing a row in Pinot table. */
+  private static GenericRow createTableRecord(int intValue, String stringValue, Object jsonValue) {
+    GenericRow record = new GenericRow();
+    record.putValue(INT_COLUMN, intValue);
+    record.putValue(STRING_COLUMN, stringValue);
+    record.putValue(JSON_COLUMN, jsonValue);
+
+    return record;
+  }
+
+  private static Map<String, String> createMapField(Pair<String, String>[] pairs) {
+    Map<String, String> map = new HashMap<>();
+    for (Pair<String, String> pair : pairs) {
+      map.put(pair.getFirst(), pair.getSecond());
+    }
+    return map;
+  }
+
+  private static org.apache.avro.Schema createRecordSchema() {
+    List<org.apache.avro.Schema.Field> fields = new ArrayList<>();
+    fields.add(new org.apache.avro.Schema.Field("id", create(Type.INT)));
+    fields.add(new org.apache.avro.Schema.Field("name", create(Type.STRING)));
+    return createRecord("record", "doc", JsonIngestionFromAvroQueriesTest.class.getCanonicalName(), false, fields);
+  }
+
+  private static GenericData.Record createRecordField(String k1, int v1, String k2, String v2) {
+    GenericData.Record record = new GenericData.Record(createRecordSchema());
+    record.put(k1, v1);
+    record.put(k2, v2);
+    return record;
+  }
+
+  private static void createInputFile()
+      throws IOException {
+    INDEX_DIR.mkdir();
+    org.apache.avro.Schema avroSchema = org.apache.avro.Schema.createRecord("eventsRecord", null, null, false);
+    List<Field> fields = Arrays
+        .asList(new Field(INT_COLUMN, createUnion(Lists.newArrayList(create(Type.INT), create(Type.NULL))), null, null),
+            new Field(STRING_COLUMN, createUnion(Lists.newArrayList(create(Type.STRING), create(Type.NULL))), null,
+                null), new Field(JSON_COLUMN,
+                createUnion(createArray(create(Type.STRING)), createMap(create(Type.STRING)), createRecordSchema(),
+                    create(Type.STRING), create(Type.NULL))));
+    avroSchema.setFields(fields);
+    List<GenericRow> inputRecords = new ArrayList<>();
+    // Insert ARRAY
+    inputRecords.add(createTableRecord(1, "daffy duck", Arrays.asList("this", "is", "a", "test")));
+
+    // Insert MAP
+    inputRecords
+        .add(createTableRecord(2, "mickey mouse", createMapField(new Pair[]{new Pair("a", "1"), new Pair("b", "2")})));
+    inputRecords
+        .add(createTableRecord(3, "donald duck", createMapField(new Pair[]{new Pair("a", "1"), new Pair("b", "2")})));
+    inputRecords.add(
+        createTableRecord(4, "scrooge mcduck", createMapField(new Pair[]{new Pair("a", "1"), new Pair("b", "2")})));
+
+    // insert RECORD
+    inputRecords.add(createTableRecord(5, "minney mouse", createRecordField("id", 1, "name", "minney")));
+
+    // Insert simple Java String (gets converted into JSON value)
+    inputRecords.add(createTableRecord(6, "pluto", "test"));
+
+    // Insert JSON string (gets converted into JSON document)
+    inputRecords.add(createTableRecord(7, "scooby doo", "{\"name\":\"scooby\",\"id\":7}"));
+
+    try (DataFileWriter<GenericData.Record> fileWriter = new DataFileWriter<>(new GenericDatumWriter<>(avroSchema))) {
+      fileWriter.create(avroSchema, AVRO_DATA_FILE);
+      for (GenericRow inputRecord : inputRecords) {
+        GenericData.Record record = new GenericData.Record(avroSchema);
+        record.put(INT_COLUMN, inputRecord.getValue(INT_COLUMN));
+        record.put(STRING_COLUMN, inputRecord.getValue(STRING_COLUMN));
+        record.put(JSON_COLUMN, inputRecord.getValue(JSON_COLUMN));
+        fileWriter.append(record);
+      }
+    }
+  }
+
+  private static RecordReader createRecordReader()
+      throws IOException {
+    Set<String> set = new HashSet<>();
+    set.add(INT_COLUMN);
+    set.add(STRING_COLUMN);
+    set.add(JSON_COLUMN);
+    AvroRecordReader avroRecordReader = new AvroRecordReader();
+    avroRecordReader.init(AVRO_DATA_FILE, set, null);
+    return avroRecordReader;
+  }
+
+  /** Create an AVRO file and then ingest it into Pinot while creating a JsonIndex. */
+  @BeforeClass
+  public void setUp()
+      throws Exception {
+    FileUtils.deleteDirectory(INDEX_DIR);
+    createInputFile();
+
+    List<String> jsonIndexColumns = new ArrayList<>();
+    jsonIndexColumns.add("jsonColumn");
+    TABLE_CONFIG.getIndexingConfig().setJsonIndexColumns(jsonIndexColumns);
+    SegmentGeneratorConfig segmentGeneratorConfig = new SegmentGeneratorConfig(TABLE_CONFIG, SCHEMA);
+    segmentGeneratorConfig.setTableName(RAW_TABLE_NAME);
+    segmentGeneratorConfig.setSegmentName(SEGMENT_NAME);
+    segmentGeneratorConfig.setOutDir(INDEX_DIR.getPath());
+    segmentGeneratorConfig.setInputFilePath(AVRO_DATA_FILE.getPath());
+
+    SegmentIndexCreationDriverImpl driver = new SegmentIndexCreationDriverImpl();
+    driver.init(segmentGeneratorConfig, createRecordReader());
+    driver.build();
+
+    IndexLoadingConfig indexLoadingConfig = new IndexLoadingConfig();
+    indexLoadingConfig.setTableConfig(TABLE_CONFIG);
+    indexLoadingConfig.setJsonIndexColumns(new HashSet<String>(jsonIndexColumns));
+    indexLoadingConfig.setReadMode(ReadMode.mmap);
+
+    ImmutableSegment immutableSegment =
+        ImmutableSegmentLoader.load(new File(INDEX_DIR, SEGMENT_NAME), indexLoadingConfig);
+    _indexSegment = immutableSegment;
+    _indexSegments = Arrays.asList(immutableSegment, immutableSegment);
+  }
+
+  /** Verify that we can query the JSON column that ingested ComplexType data from an AVRO file (see setUp). */
+  @Test
+  public void testSimpleSelectOnJsonColumn() {
+    try {
+      Operator operator = getOperatorForSqlQuery("select intColumn, stringColumn, jsonColumn FROM testTable limit 100");
+      IntermediateResultsBlock block = (IntermediateResultsBlock) operator.nextBlock();
+      Collection<Object[]> rows = block.getSelectionResult();
+      Assert.assertEquals(block.getDataSchema().getColumnDataType(0), DataSchema.ColumnDataType.INT);
+      Assert.assertEquals(block.getDataSchema().getColumnDataType(1), DataSchema.ColumnDataType.STRING);
+      Assert.assertEquals(block.getDataSchema().getColumnDataType(2), DataSchema.ColumnDataType.JSON);
+
+      List<String> expecteds = Arrays
+          .asList("[1, daffy duck, [\"this\",\"is\",\"a\",\"test\"]]", "[2, mickey mouse, {\"a\":\"1\",\"b\":\"2\"}]",
+              "[3, donald duck, {\"a\":\"1\",\"b\":\"2\"}]", "[4, scrooge mcduck, {\"a\":\"1\",\"b\":\"2\"}]",
+              "[5, minney mouse, {\"name\":\"minney\",\"id\":1}]", "[6, pluto, \"test\"]",
+              "[7, scooby doo, {\"name\":\"scooby\",\"id\":7}]");
+      int index = 0;
+
+      Iterator<Object[]> iterator = rows.iterator();
+      while (iterator.hasNext()) {
+        Object[] row = iterator.next();
+        System.out.println(Arrays.toString(row));
+        Assert.assertEquals(Arrays.toString(row), expecteds.get(index++));
+      }
+    } catch (IllegalStateException ise) {
+      Assert.assertTrue(true);
+    }
+  }
+
+  /** Verify simple path expression query on ingested Avro file. */
+  @Test
+  public void testJsonPathSelectOnJsonColumn() {
+    try {
+      Operator operator = getOperatorForSqlQuery("select intColumn, json_extract_scalar(jsonColumn, '$.name', "
+          + "'STRING', 'null') FROM testTable");
+      IntermediateResultsBlock block = (IntermediateResultsBlock) operator.nextBlock();
+      Collection<Object[]> rows = block.getSelectionResult();
+      Assert.assertEquals(block.getDataSchema().getColumnDataType(0), DataSchema.ColumnDataType.INT);
+      Assert.assertEquals(block.getDataSchema().getColumnDataType(1), DataSchema.ColumnDataType.STRING);
+
+      List<String> expecteds =
+          Arrays.asList("[1, null]", "[2, null]", "[3, null]", "[4, null]", "[5, minney]", "[6, null]", "[7, scooby]");
+      int index = 0;
+
+      Iterator<Object[]> iterator = rows.iterator();
+      while (iterator.hasNext()) {
+        Object[] row = iterator.next();
+        System.out.println(Arrays.toString(row));
+        Assert.assertEquals(Arrays.toString(row), expecteds.get(index++));
+      }
+    } catch (IllegalStateException ise) {
+      Assert.assertTrue(true);

Review comment:
       Fixed.

##########
File path: pinot-plugins/pinot-input-format/pinot-avro-base/src/main/java/org/apache/pinot/plugin/inputformat/avro/AvroIngestionSchemaValidator.java
##########
@@ -124,13 +124,16 @@ private void validateSchemas() {
               "The Pinot column: %s is 'single-value' column but the column: %s from input %s is 'multi-value' column.",
               columnName, avroColumnName, getInputSchemaType()));
         }
-        FieldSpec.DataType dataTypeForSVColumn = AvroUtils.extractFieldDataType(avroColumnField);
-        // check data type mismatch
-        if (fieldSpec.getDataType() != dataTypeForSVColumn) {
-          _dataTypeMismatch.addMismatchReason(String
-              .format("The Pinot column: (%s: %s) doesn't match with the column (%s: %s) in input %s schema.",
-                  columnName, fieldSpec.getDataType().name(), avroColumnName, avroColumnType.name(),
-                  getInputSchemaType()));
+
+        if (fieldSpec.getDataType() != FieldSpec.DataType.JSON) {

Review comment:
       Fixed.




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

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [pinot] codecov-commenter edited a comment on pull request #8426: Allow ingesting Avro complex types into JSON column.

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #8426:
URL: https://github.com/apache/pinot/pull/8426#issuecomment-1081210576


   # [Codecov](https://codecov.io/gh/apache/pinot/pull/8426?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#8426](https://codecov.io/gh/apache/pinot/pull/8426?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (48b300a) into [master](https://codecov.io/gh/apache/pinot/commit/dfc7ea8a91f0e5d1f99559f6b094d1ccc1dff5a4?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (dfc7ea8) will **decrease** coverage by `5.48%`.
   > The diff coverage is `36.36%`.
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #8426      +/-   ##
   ============================================
   - Coverage     69.66%   64.17%   -5.49%     
   - Complexity     4238     4280      +42     
   ============================================
     Files          1629     1615      -14     
     Lines         85140    85158      +18     
     Branches      12815    12937     +122     
   ============================================
   - Hits          59310    54648    -4662     
   - Misses        21698    26545    +4847     
   + Partials       4132     3965     -167     
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | integration1 | `?` | |
   | unittests1 | `66.99% <36.36%> (+0.07%)` | :arrow_up: |
   | unittests2 | `14.19% <0.00%> (+0.08%)` | :arrow_up: |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/pinot/pull/8426?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...inputformat/avro/AvroIngestionSchemaValidator.java](https://codecov.io/gh/apache/pinot/pull/8426/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtcGx1Z2lucy9waW5vdC1pbnB1dC1mb3JtYXQvcGlub3QtYXZyby1iYXNlL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9wbHVnaW4vaW5wdXRmb3JtYXQvYXZyby9BdnJvSW5nZXN0aW9uU2NoZW1hVmFsaWRhdG9yLmphdmE=) | `0.00% <0.00%> (ø)` | |
   | [.../pinot/plugin/inputformat/avro/AvroSchemaUtil.java](https://codecov.io/gh/apache/pinot/pull/8426/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtcGx1Z2lucy9waW5vdC1pbnB1dC1mb3JtYXQvcGlub3QtYXZyby1iYXNlL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9wbHVnaW4vaW5wdXRmb3JtYXQvYXZyby9BdnJvU2NoZW1hVXRpbC5qYXZh) | `18.18% <0.00%> (-0.57%)` | :arrow_down: |
   | [...a/org/apache/pinot/common/utils/PinotDataType.java](https://codecov.io/gh/apache/pinot/pull/8426/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vdXRpbHMvUGlub3REYXRhVHlwZS5qYXZh) | `81.06% <33.33%> (-0.37%)` | :arrow_down: |
   | [...t/local/recordtransformer/DataTypeTransformer.java](https://codecov.io/gh/apache/pinot/pull/8426/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC9yZWNvcmR0cmFuc2Zvcm1lci9EYXRhVHlwZVRyYW5zZm9ybWVyLmphdmE=) | `89.61% <100.00%> (+2.76%)` | :arrow_up: |
   | [...a/org/apache/pinot/common/metrics/MinionMeter.java](https://codecov.io/gh/apache/pinot/pull/8426/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vbWV0cmljcy9NaW5pb25NZXRlci5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...g/apache/pinot/common/metrics/ControllerMeter.java](https://codecov.io/gh/apache/pinot/pull/8426/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vbWV0cmljcy9Db250cm9sbGVyTWV0ZXIuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [.../apache/pinot/common/metrics/BrokerQueryPhase.java](https://codecov.io/gh/apache/pinot/pull/8426/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vbWV0cmljcy9Ccm9rZXJRdWVyeVBoYXNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [.../apache/pinot/common/metrics/MinionQueryPhase.java](https://codecov.io/gh/apache/pinot/pull/8426/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vbWV0cmljcy9NaW5pb25RdWVyeVBoYXNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...ache/pinot/server/access/AccessControlFactory.java](https://codecov.io/gh/apache/pinot/pull/8426/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VydmVyL2FjY2Vzcy9BY2Nlc3NDb250cm9sRmFjdG9yeS5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...he/pinot/common/messages/SegmentReloadMessage.java](https://codecov.io/gh/apache/pinot/pull/8426/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vbWVzc2FnZXMvU2VnbWVudFJlbG9hZE1lc3NhZ2UuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | ... and [510 more](https://codecov.io/gh/apache/pinot/pull/8426/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/pinot/pull/8426?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/pinot/pull/8426?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [dfc7ea8...48b300a](https://codecov.io/gh/apache/pinot/pull/8426?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   


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

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [pinot] codecov-commenter commented on pull request #8426: Allow ingesting Avro complex types into JSON column.

Posted by GitBox <gi...@apache.org>.
codecov-commenter commented on pull request #8426:
URL: https://github.com/apache/pinot/pull/8426#issuecomment-1081210576


   # [Codecov](https://codecov.io/gh/apache/pinot/pull/8426?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#8426](https://codecov.io/gh/apache/pinot/pull/8426?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (f8773fd) into [master](https://codecov.io/gh/apache/pinot/commit/dfc7ea8a91f0e5d1f99559f6b094d1ccc1dff5a4?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (dfc7ea8) will **decrease** coverage by `5.54%`.
   > The diff coverage is `21.05%`.
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #8426      +/-   ##
   ============================================
   - Coverage     69.66%   64.11%   -5.55%     
   - Complexity     4238     4280      +42     
   ============================================
     Files          1629     1611      -18     
     Lines         85140    84816     -324     
     Branches      12815    12875      +60     
   ============================================
   - Hits          59310    54379    -4931     
   - Misses        21698    26496    +4798     
   + Partials       4132     3941     -191     
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | integration1 | `?` | |
   | unittests1 | `66.99% <21.05%> (+0.07%)` | :arrow_up: |
   | unittests2 | `14.16% <0.00%> (+0.04%)` | :arrow_up: |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/pinot/pull/8426?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...inputformat/avro/AvroIngestionSchemaValidator.java](https://codecov.io/gh/apache/pinot/pull/8426/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtcGx1Z2lucy9waW5vdC1pbnB1dC1mb3JtYXQvcGlub3QtYXZyby1iYXNlL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9wbHVnaW4vaW5wdXRmb3JtYXQvYXZyby9BdnJvSW5nZXN0aW9uU2NoZW1hVmFsaWRhdG9yLmphdmE=) | `0.00% <0.00%> (ø)` | |
   | [.../pinot/plugin/inputformat/avro/AvroSchemaUtil.java](https://codecov.io/gh/apache/pinot/pull/8426/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtcGx1Z2lucy9waW5vdC1pbnB1dC1mb3JtYXQvcGlub3QtYXZyby1iYXNlL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9wbHVnaW4vaW5wdXRmb3JtYXQvYXZyby9BdnJvU2NoZW1hVXRpbC5qYXZh) | `17.14% <0.00%> (-1.61%)` | :arrow_down: |
   | [...a/org/apache/pinot/common/utils/PinotDataType.java](https://codecov.io/gh/apache/pinot/pull/8426/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vdXRpbHMvUGlub3REYXRhVHlwZS5qYXZh) | `81.06% <33.33%> (-0.37%)` | :arrow_down: |
   | [...t/local/recordtransformer/DataTypeTransformer.java](https://codecov.io/gh/apache/pinot/pull/8426/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC9yZWNvcmR0cmFuc2Zvcm1lci9EYXRhVHlwZVRyYW5zZm9ybWVyLmphdmE=) | `89.61% <100.00%> (+2.76%)` | :arrow_up: |
   | [...a/org/apache/pinot/common/metrics/MinionMeter.java](https://codecov.io/gh/apache/pinot/pull/8426/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vbWV0cmljcy9NaW5pb25NZXRlci5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...g/apache/pinot/common/metrics/ControllerMeter.java](https://codecov.io/gh/apache/pinot/pull/8426/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vbWV0cmljcy9Db250cm9sbGVyTWV0ZXIuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [.../apache/pinot/common/metrics/BrokerQueryPhase.java](https://codecov.io/gh/apache/pinot/pull/8426/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vbWV0cmljcy9Ccm9rZXJRdWVyeVBoYXNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [.../apache/pinot/common/metrics/MinionQueryPhase.java](https://codecov.io/gh/apache/pinot/pull/8426/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vbWV0cmljcy9NaW5pb25RdWVyeVBoYXNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...ache/pinot/server/access/AccessControlFactory.java](https://codecov.io/gh/apache/pinot/pull/8426/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VydmVyL2FjY2Vzcy9BY2Nlc3NDb250cm9sRmFjdG9yeS5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...he/pinot/common/messages/SegmentReloadMessage.java](https://codecov.io/gh/apache/pinot/pull/8426/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vbWVzc2FnZXMvU2VnbWVudFJlbG9hZE1lc3NhZ2UuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | ... and [506 more](https://codecov.io/gh/apache/pinot/pull/8426/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/pinot/pull/8426?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/pinot/pull/8426?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [dfc7ea8...f8773fd](https://codecov.io/gh/apache/pinot/pull/8426?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   


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

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [pinot] codecov-commenter edited a comment on pull request #8426: Allow ingesting Avro complex types into JSON column.

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #8426:
URL: https://github.com/apache/pinot/pull/8426#issuecomment-1081210576


   # [Codecov](https://codecov.io/gh/apache/pinot/pull/8426?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#8426](https://codecov.io/gh/apache/pinot/pull/8426?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (d338777) into [master](https://codecov.io/gh/apache/pinot/commit/dfc7ea8a91f0e5d1f99559f6b094d1ccc1dff5a4?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (dfc7ea8) will **decrease** coverage by `2.64%`.
   > The diff coverage is `36.36%`.
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #8426      +/-   ##
   ============================================
   - Coverage     69.66%   67.01%   -2.65%     
   + Complexity     4238     4197      -41     
   ============================================
     Files          1629     1258     -371     
     Lines         85140    63522   -21618     
     Branches      12815     9941    -2874     
   ============================================
   - Hits          59310    42569   -16741     
   + Misses        21698    17907    -3791     
   + Partials       4132     3046    -1086     
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | integration1 | `?` | |
   | unittests1 | `67.01% <36.36%> (+0.08%)` | :arrow_up: |
   | unittests2 | `?` | |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/pinot/pull/8426?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...inputformat/avro/AvroIngestionSchemaValidator.java](https://codecov.io/gh/apache/pinot/pull/8426/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtcGx1Z2lucy9waW5vdC1pbnB1dC1mb3JtYXQvcGlub3QtYXZyby1iYXNlL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9wbHVnaW4vaW5wdXRmb3JtYXQvYXZyby9BdnJvSW5nZXN0aW9uU2NoZW1hVmFsaWRhdG9yLmphdmE=) | `0.00% <0.00%> (ø)` | |
   | [.../pinot/plugin/inputformat/avro/AvroSchemaUtil.java](https://codecov.io/gh/apache/pinot/pull/8426/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtcGx1Z2lucy9waW5vdC1pbnB1dC1mb3JtYXQvcGlub3QtYXZyby1iYXNlL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9wbHVnaW4vaW5wdXRmb3JtYXQvYXZyby9BdnJvU2NoZW1hVXRpbC5qYXZh) | `18.18% <0.00%> (-0.57%)` | :arrow_down: |
   | [...a/org/apache/pinot/common/utils/PinotDataType.java](https://codecov.io/gh/apache/pinot/pull/8426/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vdXRpbHMvUGlub3REYXRhVHlwZS5qYXZh) | `81.06% <33.33%> (-0.37%)` | :arrow_down: |
   | [...t/local/recordtransformer/DataTypeTransformer.java](https://codecov.io/gh/apache/pinot/pull/8426/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC9yZWNvcmR0cmFuc2Zvcm1lci9EYXRhVHlwZVRyYW5zZm9ybWVyLmphdmE=) | `89.61% <100.00%> (+2.76%)` | :arrow_up: |
   | [...a/org/apache/pinot/common/metrics/MinionMeter.java](https://codecov.io/gh/apache/pinot/pull/8426/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vbWV0cmljcy9NaW5pb25NZXRlci5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...g/apache/pinot/common/metrics/ControllerMeter.java](https://codecov.io/gh/apache/pinot/pull/8426/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vbWV0cmljcy9Db250cm9sbGVyTWV0ZXIuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [.../apache/pinot/common/metrics/BrokerQueryPhase.java](https://codecov.io/gh/apache/pinot/pull/8426/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vbWV0cmljcy9Ccm9rZXJRdWVyeVBoYXNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [.../apache/pinot/common/metrics/MinionQueryPhase.java](https://codecov.io/gh/apache/pinot/pull/8426/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vbWV0cmljcy9NaW5pb25RdWVyeVBoYXNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...ache/pinot/server/access/AccessControlFactory.java](https://codecov.io/gh/apache/pinot/pull/8426/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VydmVyL2FjY2Vzcy9BY2Nlc3NDb250cm9sRmFjdG9yeS5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...he/pinot/common/messages/SegmentReloadMessage.java](https://codecov.io/gh/apache/pinot/pull/8426/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vbWVzc2FnZXMvU2VnbWVudFJlbG9hZE1lc3NhZ2UuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | ... and [747 more](https://codecov.io/gh/apache/pinot/pull/8426/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/pinot/pull/8426?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/pinot/pull/8426?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [dfc7ea8...d338777](https://codecov.io/gh/apache/pinot/pull/8426?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   


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

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [pinot] codecov-commenter edited a comment on pull request #8426: Allow ingesting Avro complex types into JSON column.

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #8426:
URL: https://github.com/apache/pinot/pull/8426#issuecomment-1081210576


   # [Codecov](https://codecov.io/gh/apache/pinot/pull/8426?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#8426](https://codecov.io/gh/apache/pinot/pull/8426?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (03cb593) into [master](https://codecov.io/gh/apache/pinot/commit/dfc7ea8a91f0e5d1f99559f6b094d1ccc1dff5a4?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (dfc7ea8) will **decrease** coverage by `55.50%`.
   > The diff coverage is `28.06%`.
   
   > :exclamation: Current head 03cb593 differs from pull request most recent head 34d8299. Consider uploading reports for the commit 34d8299 to get more accurate results
   
   ```diff
   @@              Coverage Diff              @@
   ##             master    #8426       +/-   ##
   =============================================
   - Coverage     69.66%   14.15%   -55.51%     
   + Complexity     4238       84     -4154     
   =============================================
     Files          1629     1616       -13     
     Lines         85140    85163       +23     
     Branches      12815    12933      +118     
   =============================================
   - Hits          59310    12059    -47251     
   - Misses        21698    72193    +50495     
   + Partials       4132      911     -3221     
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | integration1 | `?` | |
   | unittests1 | `?` | |
   | unittests2 | `14.15% <28.06%> (+0.04%)` | :arrow_up: |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/pinot/pull/8426?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...e/pinot/broker/api/resources/PinotBrokerDebug.java](https://codecov.io/gh/apache/pinot/pull/8426/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtYnJva2VyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9icm9rZXIvYXBpL3Jlc291cmNlcy9QaW5vdEJyb2tlckRlYnVnLmphdmE=) | `0.00% <ø> (-76.48%)` | :arrow_down: |
   | [...pinot/broker/api/resources/PinotBrokerRouting.java](https://codecov.io/gh/apache/pinot/pull/8426/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtYnJva2VyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9icm9rZXIvYXBpL3Jlc291cmNlcy9QaW5vdEJyb2tlclJvdXRpbmcuamF2YQ==) | `0.00% <ø> (ø)` | |
   | [.../BrokerResourceOnlineOfflineStateModelFactory.java](https://codecov.io/gh/apache/pinot/pull/8426/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtYnJva2VyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9icm9rZXIvYnJva2VyL2hlbGl4L0Jyb2tlclJlc291cmNlT25saW5lT2ZmbGluZVN0YXRlTW9kZWxGYWN0b3J5LmphdmE=) | `41.86% <ø> (-13.96%)` | :arrow_down: |
   | [.../helix/BrokerUserDefinedMessageHandlerFactory.java](https://codecov.io/gh/apache/pinot/pull/8426/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtYnJva2VyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9icm9rZXIvYnJva2VyL2hlbGl4L0Jyb2tlclVzZXJEZWZpbmVkTWVzc2FnZUhhbmRsZXJGYWN0b3J5LmphdmE=) | `33.96% <ø> (-35.85%)` | :arrow_down: |
   | [...roker/requesthandler/GrpcBrokerRequestHandler.java](https://codecov.io/gh/apache/pinot/pull/8426/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtYnJva2VyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9icm9rZXIvcmVxdWVzdGhhbmRsZXIvR3JwY0Jyb2tlclJlcXVlc3RIYW5kbGVyLmphdmE=) | `0.00% <0.00%> (ø)` | |
   | [...va/org/apache/pinot/common/config/NettyConfig.java](https://codecov.io/gh/apache/pinot/pull/8426/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vY29uZmlnL05ldHR5Q29uZmlnLmphdmE=) | `0.00% <0.00%> (ø)` | |
   | [...pache/pinot/common/config/provider/TableCache.java](https://codecov.io/gh/apache/pinot/pull/8426/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vY29uZmlnL3Byb3ZpZGVyL1RhYmxlQ2FjaGUuamF2YQ==) | `0.00% <0.00%> (-76.57%)` | :arrow_down: |
   | [.../pinot/common/function/DateTimePatternHandler.java](https://codecov.io/gh/apache/pinot/pull/8426/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vZnVuY3Rpb24vRGF0ZVRpbWVQYXR0ZXJuSGFuZGxlci5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...org/apache/pinot/common/function/FunctionInfo.java](https://codecov.io/gh/apache/pinot/pull/8426/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vZnVuY3Rpb24vRnVuY3Rpb25JbmZvLmphdmE=) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...apache/pinot/common/function/FunctionRegistry.java](https://codecov.io/gh/apache/pinot/pull/8426/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vZnVuY3Rpb24vRnVuY3Rpb25SZWdpc3RyeS5qYXZh) | `0.00% <0.00%> (-87.10%)` | :arrow_down: |
   | ... and [1465 more](https://codecov.io/gh/apache/pinot/pull/8426/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/pinot/pull/8426?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/pinot/pull/8426?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [dfc7ea8...34d8299](https://codecov.io/gh/apache/pinot/pull/8426?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   


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

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [pinot] siddharthteotia merged pull request #8426: Allow ingesting Avro complex types into JSON column.

Posted by GitBox <gi...@apache.org>.
siddharthteotia merged pull request #8426:
URL: https://github.com/apache/pinot/pull/8426


   


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

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org