siddharthteotia commented on a change in pull request #6525:
URL: https://github.com/apache/incubator-pinot/pull/6525#discussion_r569203392



##########
File path: 
pinot-plugins/pinot-input-format/pinot-parquet/src/main/java/org/apache/pinot/plugin/inputformat/parquet/ParquetNativeRecordExtractor.java
##########
@@ -0,0 +1,263 @@
+/**
+ * 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.plugin.inputformat.parquet;
+
+import com.google.common.collect.ImmutableSet;
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import javax.annotation.Nullable;
+import org.apache.parquet.example.data.Group;
+import org.apache.parquet.io.api.Binary;
+import org.apache.parquet.schema.DecimalMetadata;
+import org.apache.parquet.schema.OriginalType;
+import org.apache.parquet.schema.Type;
+import org.apache.pinot.spi.data.readers.BaseRecordExtractor;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.data.readers.RecordExtractorConfig;
+import org.joda.time.DateTimeConstants;
+
+import static java.lang.Math.pow;
+
+
+public class ParquetNativeRecordExtractor extends BaseRecordExtractor<Group> {
+
+  /**
+   * Number of days between Julian day epoch (January 1, 4713 BC) and Unix day 
epoch (January 1, 1970).
+   * The value of this constant is {@value}.
+   */
+  public static final long JULIAN_DAY_NUMBER_FOR_UNIX_EPOCH = 2440588;
+
+  public static final long NANOS_PER_MILLISECOND = 1000000;
+
+  private Set<String> _fields;
+  private boolean _extractAll = false;
+
+  @Override
+  public void init(@Nullable Set<String> fields, RecordExtractorConfig 
recordExtractorConfig) {
+    if (fields == null || fields.isEmpty()) {
+      _extractAll = true;
+      _fields = Collections.emptySet();
+    } else {
+      _fields = ImmutableSet.copyOf(fields);
+    }
+  }
+
+  @Override
+  public GenericRow extract(Group from, GenericRow to) {
+    if (_extractAll) {
+      List<Type> fields = from.getType().getFields();
+      for (Type field : fields) {
+        String fieldName = field.getName();
+        Object value = extractValue(from, 
from.getType().getFieldIndex(fieldName));
+        if (value != null) {
+          value = convert(value);
+        }
+        to.putValue(fieldName, value);
+      }
+    } else {
+      for (String fieldName : _fields) {
+        Object value = extractValue(from, 
from.getType().getFieldIndex(fieldName));
+        if (value != null) {
+          value = convert(value);
+        }
+        to.putValue(fieldName, value);
+      }
+    }
+    return to;
+  }
+
+  private Object extractValue(Group from, int field) {
+    int valueCount = from.getFieldRepetitionCount(field);
+    Type fieldType = from.getType().getType(field);
+    if (valueCount == 0) {
+      return null;
+    }
+    if (valueCount == 1) {
+      return extractValue(from, field, fieldType, 0);
+    }
+    Object[] results = new Object[valueCount];
+    for (int index = 0; index < valueCount; index++) {
+      results[index] = extractValue(from, field, fieldType, index);
+    }
+    return results;
+  }
+
+  private Object extractValue(Group from, int field, Type fieldType, int 
index) {
+    String valueToString = from.getValueToString(field, index);
+    if (fieldType.isPrimitive()) {
+      switch (fieldType.asPrimitiveType().getPrimitiveTypeName()) {
+        case INT32:
+          return from.getInteger(field, index);
+        case INT64:
+          return from.getLong(field, index);
+        case FLOAT:
+          return from.getFloat(field, index);
+        case DOUBLE:
+          return from.getDouble(field, index);
+        case BOOLEAN:
+          return from.getValueToString(field, index);
+        case BINARY:
+        case FIXED_LEN_BYTE_ARRAY:
+          final OriginalType originalType = fieldType.getOriginalType();
+          if (fieldType.getOriginalType() == OriginalType.UTF8) {
+            return from.getValueToString(field, index);
+          }
+          if (fieldType.getOriginalType() == OriginalType.DECIMAL) {
+            DecimalMetadata decimalMetadata = 
fieldType.asPrimitiveType().getDecimalMetadata();
+            return binaryToDecimal(from.getBinary(field, index), 
decimalMetadata.getPrecision(),
+                decimalMetadata.getScale());
+          }
+          return from.getBinary(field, index);
+        case INT96:
+          Binary int96 = from.getInt96(field, index);
+          ByteBuffer buf = 
ByteBuffer.wrap(int96.getBytes()).order(ByteOrder.LITTLE_ENDIAN);
+          long dateTime = (buf.getInt(8) - JULIAN_DAY_NUMBER_FOR_UNIX_EPOCH) * 
DateTimeConstants.MILLIS_PER_DAY
+              + buf.getLong(0) / NANOS_PER_MILLISECOND;
+          return dateTime;
+      }
+    } else if (fieldType.isRepetition(Type.Repetition.OPTIONAL)) {
+      Group group = from.getGroup(field, index);
+      if (fieldType.getOriginalType() == OriginalType.LIST) {
+        return extractList(group);
+      }
+      return extractMap(group);
+    } else if (fieldType.isRepetition(Type.Repetition.REPEATED)) {
+      Group group = from.getGroup(field, index);
+      return extractMap(group);
+    } else if (fieldType.isRepetition(Type.Repetition.REQUIRED)) {
+      Group group = from.getGroup(field, index);

Review comment:
       this code is same as for OPTIONAL ? 

##########
File path: 
pinot-plugins/pinot-input-format/pinot-parquet/src/main/java/org/apache/pinot/plugin/inputformat/parquet/ParquetNativeRecordExtractor.java
##########
@@ -0,0 +1,263 @@
+/**
+ * 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.plugin.inputformat.parquet;
+
+import com.google.common.collect.ImmutableSet;
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import javax.annotation.Nullable;
+import org.apache.parquet.example.data.Group;
+import org.apache.parquet.io.api.Binary;
+import org.apache.parquet.schema.DecimalMetadata;
+import org.apache.parquet.schema.OriginalType;
+import org.apache.parquet.schema.Type;
+import org.apache.pinot.spi.data.readers.BaseRecordExtractor;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.data.readers.RecordExtractorConfig;
+import org.joda.time.DateTimeConstants;
+
+import static java.lang.Math.pow;
+
+
+public class ParquetNativeRecordExtractor extends BaseRecordExtractor<Group> {
+
+  /**
+   * Number of days between Julian day epoch (January 1, 4713 BC) and Unix day 
epoch (January 1, 1970).
+   * The value of this constant is {@value}.
+   */
+  public static final long JULIAN_DAY_NUMBER_FOR_UNIX_EPOCH = 2440588;
+
+  public static final long NANOS_PER_MILLISECOND = 1000000;
+
+  private Set<String> _fields;
+  private boolean _extractAll = false;
+
+  @Override
+  public void init(@Nullable Set<String> fields, RecordExtractorConfig 
recordExtractorConfig) {
+    if (fields == null || fields.isEmpty()) {
+      _extractAll = true;
+      _fields = Collections.emptySet();
+    } else {
+      _fields = ImmutableSet.copyOf(fields);
+    }
+  }
+
+  @Override
+  public GenericRow extract(Group from, GenericRow to) {
+    if (_extractAll) {
+      List<Type> fields = from.getType().getFields();
+      for (Type field : fields) {
+        String fieldName = field.getName();
+        Object value = extractValue(from, 
from.getType().getFieldIndex(fieldName));
+        if (value != null) {
+          value = convert(value);
+        }
+        to.putValue(fieldName, value);
+      }
+    } else {
+      for (String fieldName : _fields) {
+        Object value = extractValue(from, 
from.getType().getFieldIndex(fieldName));
+        if (value != null) {
+          value = convert(value);
+        }
+        to.putValue(fieldName, value);
+      }
+    }
+    return to;
+  }
+
+  private Object extractValue(Group from, int field) {
+    int valueCount = from.getFieldRepetitionCount(field);
+    Type fieldType = from.getType().getType(field);
+    if (valueCount == 0) {
+      return null;
+    }
+    if (valueCount == 1) {
+      return extractValue(from, field, fieldType, 0);
+    }
+    Object[] results = new Object[valueCount];
+    for (int index = 0; index < valueCount; index++) {
+      results[index] = extractValue(from, field, fieldType, index);
+    }
+    return results;
+  }
+
+  private Object extractValue(Group from, int field, Type fieldType, int 
index) {
+    String valueToString = from.getValueToString(field, index);
+    if (fieldType.isPrimitive()) {
+      switch (fieldType.asPrimitiveType().getPrimitiveTypeName()) {
+        case INT32:
+          return from.getInteger(field, index);
+        case INT64:
+          return from.getLong(field, index);
+        case FLOAT:
+          return from.getFloat(field, index);
+        case DOUBLE:
+          return from.getDouble(field, index);
+        case BOOLEAN:
+          return from.getValueToString(field, index);
+        case BINARY:
+        case FIXED_LEN_BYTE_ARRAY:
+          final OriginalType originalType = fieldType.getOriginalType();
+          if (fieldType.getOriginalType() == OriginalType.UTF8) {
+            return from.getValueToString(field, index);
+          }
+          if (fieldType.getOriginalType() == OriginalType.DECIMAL) {
+            DecimalMetadata decimalMetadata = 
fieldType.asPrimitiveType().getDecimalMetadata();
+            return binaryToDecimal(from.getBinary(field, index), 
decimalMetadata.getPrecision(),
+                decimalMetadata.getScale());
+          }
+          return from.getBinary(field, index);
+        case INT96:
+          Binary int96 = from.getInt96(field, index);
+          ByteBuffer buf = 
ByteBuffer.wrap(int96.getBytes()).order(ByteOrder.LITTLE_ENDIAN);
+          long dateTime = (buf.getInt(8) - JULIAN_DAY_NUMBER_FOR_UNIX_EPOCH) * 
DateTimeConstants.MILLIS_PER_DAY
+              + buf.getLong(0) / NANOS_PER_MILLISECOND;
+          return dateTime;
+      }
+    } else if (fieldType.isRepetition(Type.Repetition.OPTIONAL)) {
+      Group group = from.getGroup(field, index);
+      if (fieldType.getOriginalType() == OriginalType.LIST) {
+        return extractList(group);
+      }
+      return extractMap(group);
+    } else if (fieldType.isRepetition(Type.Repetition.REPEATED)) {

Review comment:
       If the field has repetition, shouldn't we use extractList() ?

##########
File path: 
pinot-plugins/pinot-input-format/pinot-parquet/src/main/java/org/apache/pinot/plugin/inputformat/parquet/ParquetNativeRecordExtractor.java
##########
@@ -0,0 +1,263 @@
+/**
+ * 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.plugin.inputformat.parquet;
+
+import com.google.common.collect.ImmutableSet;
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import javax.annotation.Nullable;
+import org.apache.parquet.example.data.Group;
+import org.apache.parquet.io.api.Binary;
+import org.apache.parquet.schema.DecimalMetadata;
+import org.apache.parquet.schema.OriginalType;
+import org.apache.parquet.schema.Type;
+import org.apache.pinot.spi.data.readers.BaseRecordExtractor;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.data.readers.RecordExtractorConfig;
+import org.joda.time.DateTimeConstants;
+
+import static java.lang.Math.pow;
+
+
+public class ParquetNativeRecordExtractor extends BaseRecordExtractor<Group> {
+
+  /**
+   * Number of days between Julian day epoch (January 1, 4713 BC) and Unix day 
epoch (January 1, 1970).
+   * The value of this constant is {@value}.
+   */
+  public static final long JULIAN_DAY_NUMBER_FOR_UNIX_EPOCH = 2440588;
+
+  public static final long NANOS_PER_MILLISECOND = 1000000;
+
+  private Set<String> _fields;
+  private boolean _extractAll = false;
+
+  @Override
+  public void init(@Nullable Set<String> fields, RecordExtractorConfig 
recordExtractorConfig) {
+    if (fields == null || fields.isEmpty()) {
+      _extractAll = true;
+      _fields = Collections.emptySet();
+    } else {
+      _fields = ImmutableSet.copyOf(fields);
+    }
+  }
+
+  @Override
+  public GenericRow extract(Group from, GenericRow to) {
+    if (_extractAll) {
+      List<Type> fields = from.getType().getFields();
+      for (Type field : fields) {
+        String fieldName = field.getName();
+        Object value = extractValue(from, 
from.getType().getFieldIndex(fieldName));
+        if (value != null) {
+          value = convert(value);
+        }
+        to.putValue(fieldName, value);
+      }
+    } else {
+      for (String fieldName : _fields) {
+        Object value = extractValue(from, 
from.getType().getFieldIndex(fieldName));
+        if (value != null) {
+          value = convert(value);
+        }
+        to.putValue(fieldName, value);
+      }
+    }
+    return to;
+  }
+
+  private Object extractValue(Group from, int field) {
+    int valueCount = from.getFieldRepetitionCount(field);
+    Type fieldType = from.getType().getType(field);
+    if (valueCount == 0) {
+      return null;
+    }
+    if (valueCount == 1) {
+      return extractValue(from, field, fieldType, 0);
+    }
+    Object[] results = new Object[valueCount];
+    for (int index = 0; index < valueCount; index++) {
+      results[index] = extractValue(from, field, fieldType, index);
+    }
+    return results;
+  }
+
+  private Object extractValue(Group from, int field, Type fieldType, int 
index) {

Review comment:
       (nit) consider renaming field to fieldIndex or columnIndex and index to 
repetitionValueIndex or something similar.
   
   index doesn't represent rowIndex right? I believe it comes from 
fieldRepetitionCount?

##########
File path: 
pinot-plugins/pinot-input-format/pinot-parquet/src/main/java/org/apache/pinot/plugin/inputformat/parquet/ParquetNativeRecordExtractor.java
##########
@@ -0,0 +1,263 @@
+/**
+ * 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.plugin.inputformat.parquet;
+
+import com.google.common.collect.ImmutableSet;
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import javax.annotation.Nullable;
+import org.apache.parquet.example.data.Group;
+import org.apache.parquet.io.api.Binary;
+import org.apache.parquet.schema.DecimalMetadata;
+import org.apache.parquet.schema.OriginalType;
+import org.apache.parquet.schema.Type;
+import org.apache.pinot.spi.data.readers.BaseRecordExtractor;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.data.readers.RecordExtractorConfig;
+import org.joda.time.DateTimeConstants;
+
+import static java.lang.Math.pow;
+
+
+public class ParquetNativeRecordExtractor extends BaseRecordExtractor<Group> {
+
+  /**
+   * Number of days between Julian day epoch (January 1, 4713 BC) and Unix day 
epoch (January 1, 1970).
+   * The value of this constant is {@value}.
+   */
+  public static final long JULIAN_DAY_NUMBER_FOR_UNIX_EPOCH = 2440588;
+
+  public static final long NANOS_PER_MILLISECOND = 1000000;
+
+  private Set<String> _fields;
+  private boolean _extractAll = false;
+
+  @Override
+  public void init(@Nullable Set<String> fields, RecordExtractorConfig 
recordExtractorConfig) {
+    if (fields == null || fields.isEmpty()) {
+      _extractAll = true;
+      _fields = Collections.emptySet();
+    } else {
+      _fields = ImmutableSet.copyOf(fields);
+    }
+  }
+
+  @Override
+  public GenericRow extract(Group from, GenericRow to) {
+    if (_extractAll) {
+      List<Type> fields = from.getType().getFields();
+      for (Type field : fields) {
+        String fieldName = field.getName();
+        Object value = extractValue(from, 
from.getType().getFieldIndex(fieldName));
+        if (value != null) {
+          value = convert(value);
+        }
+        to.putValue(fieldName, value);
+      }
+    } else {
+      for (String fieldName : _fields) {
+        Object value = extractValue(from, 
from.getType().getFieldIndex(fieldName));
+        if (value != null) {
+          value = convert(value);
+        }
+        to.putValue(fieldName, value);
+      }
+    }
+    return to;
+  }
+
+  private Object extractValue(Group from, int field) {

Review comment:
       (nit) consider renaming field to fieldIndex or columnIndex

##########
File path: 
pinot-plugins/pinot-input-format/pinot-parquet/src/main/java/org/apache/pinot/plugin/inputformat/parquet/ParquetNativeRecordExtractor.java
##########
@@ -0,0 +1,263 @@
+/**
+ * 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.plugin.inputformat.parquet;
+
+import com.google.common.collect.ImmutableSet;
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import javax.annotation.Nullable;
+import org.apache.parquet.example.data.Group;
+import org.apache.parquet.io.api.Binary;
+import org.apache.parquet.schema.DecimalMetadata;
+import org.apache.parquet.schema.OriginalType;
+import org.apache.parquet.schema.Type;
+import org.apache.pinot.spi.data.readers.BaseRecordExtractor;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.data.readers.RecordExtractorConfig;
+import org.joda.time.DateTimeConstants;
+
+import static java.lang.Math.pow;
+
+
+public class ParquetNativeRecordExtractor extends BaseRecordExtractor<Group> {
+
+  /**
+   * Number of days between Julian day epoch (January 1, 4713 BC) and Unix day 
epoch (January 1, 1970).
+   * The value of this constant is {@value}.
+   */
+  public static final long JULIAN_DAY_NUMBER_FOR_UNIX_EPOCH = 2440588;
+
+  public static final long NANOS_PER_MILLISECOND = 1000000;
+
+  private Set<String> _fields;
+  private boolean _extractAll = false;
+
+  @Override
+  public void init(@Nullable Set<String> fields, RecordExtractorConfig 
recordExtractorConfig) {
+    if (fields == null || fields.isEmpty()) {
+      _extractAll = true;
+      _fields = Collections.emptySet();
+    } else {
+      _fields = ImmutableSet.copyOf(fields);
+    }
+  }
+
+  @Override
+  public GenericRow extract(Group from, GenericRow to) {

Review comment:
       Although this is not on performance critical path, I think parquet does 
provide efficient columnar readers. We should consider.
   
   Also, this interface is not clear. The extraction is happening one Parquet 
record at a time but the method takes a RowGroup?

##########
File path: 
pinot-plugins/pinot-input-format/pinot-parquet/src/main/java/org/apache/pinot/plugin/inputformat/parquet/ParquetAvroRecordReader.java
##########
@@ -0,0 +1,85 @@
+/**
+ * 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.plugin.inputformat.parquet;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Set;
+import javax.annotation.Nullable;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.hadoop.fs.Path;
+import org.apache.parquet.hadoop.ParquetReader;
+import org.apache.pinot.plugin.inputformat.avro.AvroRecordExtractor;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.data.readers.RecordReader;
+import org.apache.pinot.spi.data.readers.RecordReaderConfig;
+
+
+/**
+ * Record reader for Parquet file.
+ */
+public class ParquetAvroRecordReader implements RecordReader {

Review comment:
       If the underlying file is Parquet, why are we using AvroRecordExtractor 
in this reader? 
   Also, the class name is confusing.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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

Reply via email to