arina-ielchiieva commented on a change in pull request #1951: DRILL-7454: Convert Avro to EVF URL: https://github.com/apache/drill/pull/1951#discussion_r363756143
########## File path: exec/java-exec/src/main/java/org/apache/drill/exec/store/avro/AvroBatchReader.java ########## @@ -0,0 +1,368 @@ +/* + * 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.drill.exec.store.avro; + +import org.apache.avro.Schema; +import org.apache.avro.file.DataFileReader; +import org.apache.avro.generic.GenericArray; +import org.apache.avro.generic.GenericContainer; +import org.apache.avro.generic.GenericDatumReader; +import org.apache.avro.generic.GenericFixed; +import org.apache.avro.generic.GenericRecord; +import org.apache.avro.mapred.FsInput; +import org.apache.avro.util.Utf8; +import org.apache.drill.common.exceptions.UserException; +import org.apache.drill.exec.physical.impl.scan.file.FileScanFramework; +import org.apache.drill.exec.physical.impl.scan.framework.ManagedReader; +import org.apache.drill.exec.physical.resultSet.ResultSetLoader; +import org.apache.drill.exec.physical.resultSet.RowSetLoader; +import org.apache.drill.exec.record.metadata.ColumnMetadata; +import org.apache.drill.exec.record.metadata.TupleMetadata; +import org.apache.drill.exec.util.ImpersonationUtil; +import org.apache.drill.exec.vector.accessor.ArrayWriter; +import org.apache.drill.exec.vector.accessor.DictWriter; +import org.apache.drill.exec.vector.accessor.ObjectWriter; +import org.apache.drill.exec.vector.accessor.ScalarWriter; +import org.apache.drill.exec.vector.accessor.TupleWriter; +import org.apache.drill.shaded.guava.com.google.common.base.Charsets; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.mapred.FileSplit; +import org.apache.hadoop.security.UserGroupInformation; +import org.joda.time.DateTimeConstants; +import org.joda.time.Period; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.math.BigDecimal; +import java.math.BigInteger; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.nio.IntBuffer; +import java.security.PrivilegedExceptionAction; +import java.util.List; +import java.util.Map; + +public class AvroBatchReader implements ManagedReader<FileScanFramework.FileSchemaNegotiator> { + + private static final Logger logger = LoggerFactory.getLogger(AvroBatchReader.class); + + // currently config is unused but maybe used later + private final AvroReaderConfig config; + + private Path filePath; + private long endPosition; + private DataFileReader<GenericContainer> reader; + private ResultSetLoader loader; + // re-use container instance + private GenericContainer container = null; + + public AvroBatchReader(AvroReaderConfig config) { + this.config = config; + } + + @Override + public boolean open(FileScanFramework.FileSchemaNegotiator negotiator) { + FileSplit split = negotiator.split(); + filePath = split.getPath(); + + // Avro files are splittable, define reading start / end positions + long startPosition = split.getStart(); + endPosition = startPosition + split.getLength(); + + logger.debug("Processing Avro file: {}, start position: {}, end position: {}", + filePath, startPosition, endPosition); + + reader = prepareReader(split, negotiator.fileSystem(), + negotiator.userName(), negotiator.context().getFragmentContext().getQueryUserName()); + + logger.debug("Avro file schema: {}", reader.getSchema()); + TupleMetadata schema = AvroSchemaUtil.convert(reader.getSchema()); + logger.debug("Avro file converted schema: {}", schema); + negotiator.setTableSchema(schema, true); + loader = negotiator.build(); + + return true; + } + + @Override + public boolean next() { + RowSetLoader rowWriter = loader.writer(); + while (!rowWriter.isFull()) { + if (!nextLine(rowWriter)) { + return false; + } + } + return true; + } + + @Override + public void close() { + try { + reader.close(); + } catch (IOException e) { + logger.warn("Error closing Avro reader: {}", e.getMessage(), e); + } finally { + reader = null; + } + } + + @Override + public String toString() { + long currentPosition = -1L; + try { + if (reader != null) { + currentPosition = reader.tell(); + } + } catch (IOException e) { + logger.trace("Unable to obtain Avro reader position: {}", e.getMessage(), e); + } + return "AvroBatchReader[File=" + filePath + + ", Position=" + currentPosition + + "]"; + } + + /** + * Initialized Avro data reader based on given file system and file path. + * Moves reader to the sync point from where to start reading the data. + * + * @param fileSplit file split + * @param fs file system + * @param opUserName name of the user whom to impersonate while reading the data + * @param queryUserName name of the user who issues the query + * @return Avro file reader + */ + private DataFileReader<GenericContainer> prepareReader(FileSplit fileSplit, FileSystem fs, String opUserName, String queryUserName) { + try { + UserGroupInformation ugi = ImpersonationUtil.createProxyUgi(opUserName, queryUserName); + DataFileReader<GenericContainer> reader = ugi.doAs((PrivilegedExceptionAction<DataFileReader<GenericContainer>>) () -> + new DataFileReader<>(new FsInput(fileSplit.getPath(), fs.getConf()), new GenericDatumReader<GenericContainer>())); + + // move to sync point from where to read the file + reader.sync(fileSplit.getStart()); + return reader; + } catch (IOException | InterruptedException e) { + throw UserException.dataReadError(e) + .message("Error preparing Avro reader") + .addContext("Reader", this) + .build(logger); + } + } + + private boolean nextLine(RowSetLoader rowWriter) { + try { + if (!reader.hasNext() || reader.pastSync(endPosition)) { + return false; + } + container = reader.next(container); + } catch (IOException e) { + throw UserException.dataReadError(e) + .addContext("Reader", this) + .build(logger); + } + + Schema schema = container.getSchema(); + GenericRecord record = (GenericRecord) container; + + if (Schema.Type.RECORD != schema.getType()) { + throw UserException.dataReadError() + .message("Root object must be record type. Found: %s", schema.getType()) + .addContext("Reader", this) + .build(logger); + } + + rowWriter.start(); + List<Schema.Field> fields = schema.getFields(); + for (Schema.Field field : fields) { + String fieldName = field.name(); + Object value = record.get(fieldName); + ObjectWriter writer = rowWriter.column(fieldName); + processRecord(writer, value, field.schema()); + } + rowWriter.save(); + return true; + } + + private void processRecord(ObjectWriter writer, Object value, Schema schema) { + // skip processing record if it is null or is not projected + if (value == null || !writer.isProjected()) { + return; + } + + switch (schema.getType()) { + case UNION: + processRecord(writer, value, AvroSchemaUtil.extractSchemaFromNullable(schema, writer.schema().name())); + break; + case RECORD: + TupleWriter tupleWriter = writer.tuple(); + + if (tupleWriter.tupleSchema().isEmpty()) { Review comment: No, I mean records that refer to named records that are not constructed yet. Regular, maps / nested map are constructed easily, since we know the structure beforehand: https://github.com/arina-ielchiieva/drill/blob/DRILL-7454/exec/java-exec/src/test/java/org/apache/drill/exec/store/avro/AvroSchemaUtilTest.java#L246 The case I am referring to is a little bit more complex, in Avro documentation they just provide one example of it: ``` For example, a linked-list of 64-bit values may be defined with: { "type": "record", "name": "LongList", "aliases": ["LinkedLongs"], // old name for this "fields" : [ {"name": "value", "type": "long"}, // each element has a long {"name": "next", "type": ["null", "LongList"]} // optional next element ] } ``` Simply put they allow a field of record type to refer to another record by name. The problem occurs when one record type refers to another which is not yet constructed, moreover it cannot be constructed until upper type is constructed. Some more complicated schema examples: https://github.com/arina-ielchiieva/drill/blob/DRILL-7454/exec/java-exec/src/test/java/org/apache/drill/exec/store/avro/AvroSchemaUtilTest.java#L343 For these cases, I put off such type construction till the runtime when exact schema is know, simply adding map without fields. Explanation in the code: https://github.com/arina-ielchiieva/drill/blob/DRILL-7454/exec/java-exec/src/main/java/org/apache/drill/exec/store/avro/AvroSchemaUtil.java#L118 In reality, I believe it is a rare case but we had unit test in Drill so I had to cover this case as well. ---------------------------------------------------------------- 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: [email protected] With regards, Apache Git Services
