public void readAllFixedFields(long recordsToRead) throws IOException {
   boolean useAsyncColReader =
       fragmentContext.getOptions().getOption(ExecConstants.PARQUET_COLUMNREADER_ASYNC).bool_val;
   if (useAsyncColReader) {
     readAllFixedFieldsParallel(recordsToRead);
   } else {
     readAllFixedFieldsSerial(recordsToRead);
     ;
   }
 }
Exemplo n.º 2
0
  public RecordWriter getRecordWriter(FragmentContext context, ParquetWriter writer)
      throws IOException, OutOfMemoryException {
    Map<String, String> options = Maps.newHashMap();

    options.put("location", writer.getLocation());

    FragmentHandle handle = context.getHandle();
    String fragmentId =
        String.format("%d_%d", handle.getMajorFragmentId(), handle.getMinorFragmentId());
    options.put("prefix", fragmentId);

    options.put(
        FileSystem.FS_DEFAULT_NAME_KEY, ((FileSystemConfig) writer.getStorageConfig()).connection);

    options.put(
        ExecConstants.PARQUET_BLOCK_SIZE,
        context.getOptions().getOption(ExecConstants.PARQUET_BLOCK_SIZE).num_val.toString());
    options.put(
        ExecConstants.PARQUET_PAGE_SIZE,
        context.getOptions().getOption(ExecConstants.PARQUET_PAGE_SIZE).num_val.toString());
    options.put(
        ExecConstants.PARQUET_DICT_PAGE_SIZE,
        context.getOptions().getOption(ExecConstants.PARQUET_DICT_PAGE_SIZE).num_val.toString());

    options.put(
        ExecConstants.PARQUET_WRITER_COMPRESSION_TYPE,
        context.getOptions().getOption(ExecConstants.PARQUET_WRITER_COMPRESSION_TYPE).string_val);

    options.put(
        ExecConstants.PARQUET_WRITER_ENABLE_DICTIONARY_ENCODING,
        context
            .getOptions()
            .getOption(ExecConstants.PARQUET_WRITER_ENABLE_DICTIONARY_ENCODING)
            .bool_val
            .toString());

    RecordWriter recordWriter = new ParquetRecordWriter(context, writer);
    recordWriter.init(options);

    return recordWriter;
  }
Exemplo n.º 3
0
  public SingleBatchSorter createNewSorter(FragmentContext context, VectorAccessible batch)
      throws ClassTransformationException, IOException, SchemaChangeException {
    CodeGenerator<SingleBatchSorter> cg =
        CodeGenerator.get(
            SingleBatchSorter.TEMPLATE_DEFINITION,
            context.getFunctionRegistry(),
            context.getOptions());
    ClassGenerator<SingleBatchSorter> g = cg.getRoot();

    generateComparisons(g, batch);

    return context.getImplementationClass(cg);
  }
Exemplo n.º 4
0
  private MSorter createNewMSorter(
      FragmentContext context,
      List<Ordering> orderings,
      VectorAccessible batch,
      MappingSet mainMapping,
      MappingSet leftMapping,
      MappingSet rightMapping)
      throws ClassTransformationException, IOException, SchemaChangeException {
    CodeGenerator<MSorter> cg =
        CodeGenerator.get(
            MSorter.TEMPLATE_DEFINITION, context.getFunctionRegistry(), context.getOptions());
    ClassGenerator<MSorter> g = cg.getRoot();
    g.setMappingSet(mainMapping);

    for (Ordering od : orderings) {
      // first, we rewrite the evaluation stack for each side of the comparison.
      ErrorCollector collector = new ErrorCollectorImpl();
      final LogicalExpression expr =
          ExpressionTreeMaterializer.materialize(
              od.getExpr(), batch, collector, context.getFunctionRegistry());
      if (collector.hasErrors()) {
        throw new SchemaChangeException(
            "Failure while materializing expression. " + collector.toErrorString());
      }
      g.setMappingSet(leftMapping);
      HoldingContainer left = g.addExpr(expr, ClassGenerator.BlkCreateMode.FALSE);
      g.setMappingSet(rightMapping);
      HoldingContainer right = g.addExpr(expr, ClassGenerator.BlkCreateMode.FALSE);
      g.setMappingSet(mainMapping);

      // next we wrap the two comparison sides and add the expression block for the comparison.
      LogicalExpression fh =
          FunctionGenerationHelper.getOrderingComparator(
              od.nullsSortHigh(), left, right, context.getFunctionRegistry());
      HoldingContainer out = g.addExpr(fh, ClassGenerator.BlkCreateMode.FALSE);
      JConditional jc = g.getEvalBlock()._if(out.getValue().ne(JExpr.lit(0)));

      if (od.getDirection() == Direction.ASCENDING) {
        jc._then()._return(out.getValue());
      } else {
        jc._then()._return(out.getValue().minus());
      }
      g.rotateBlock();
    }

    g.rotateBlock();
    g.getEvalBlock()._return(JExpr.lit(0));

    return context.getImplementationClass(cg);
  }
  @Override
  public void setup(OperatorContext operatorContext, OutputMutator output)
      throws ExecutionSetupException {
    this.operatorContext = operatorContext;
    if (!isStarQuery()) {
      columnsFound = new boolean[getColumns().size()];
      nullFilledVectors = new ArrayList<>();
    }
    columnStatuses = new ArrayList<>();
    //    totalRecords = footer.getBlocks().get(rowGroupIndex).getRowCount();
    List<ColumnDescriptor> columns = footer.getFileMetaData().getSchema().getColumns();
    allFieldsFixedLength = true;
    ColumnDescriptor column;
    ColumnChunkMetaData columnChunkMetaData;
    int columnsToScan = 0;
    mockRecordsRead = 0;

    MaterializedField field;
    //    ParquetMetadataConverter metaConverter = new ParquetMetadataConverter();
    FileMetaData fileMetaData;

    logger.debug(
        "Reading row group({}) with {} records in file {}.",
        rowGroupIndex,
        footer.getBlocks().get(rowGroupIndex).getRowCount(),
        hadoopPath.toUri().getPath());
    totalRecordsRead = 0;

    // TODO - figure out how to deal with this better once we add nested reading, note also look
    // where this map is used below
    // store a map from column name to converted types if they are non-null
    Map<String, SchemaElement> schemaElements =
        ParquetReaderUtility.getColNameToSchemaElementMapping(footer);

    // loop to add up the length of the fixed width columns and build the schema
    for (int i = 0; i < columns.size(); ++i) {
      column = columns.get(i);
      SchemaElement se = schemaElements.get(column.getPath()[0]);
      MajorType mt =
          ParquetToDrillTypeConverter.toMajorType(
              column.getType(),
              se.getType_length(),
              getDataMode(column),
              se,
              fragmentContext.getOptions());
      field = MaterializedField.create(toFieldName(column.getPath()), mt);
      if (!fieldSelected(field)) {
        continue;
      }
      columnsToScan++;
      int dataTypeLength = getDataTypeLength(column, se);
      if (dataTypeLength == -1) {
        allFieldsFixedLength = false;
      } else {
        bitWidthAllFixedFields += dataTypeLength;
      }
    }
    //    rowGroupOffset =
    // footer.getBlocks().get(rowGroupIndex).getColumns().get(0).getFirstDataPageOffset();

    if (columnsToScan != 0 && allFieldsFixedLength) {
      recordsPerBatch =
          (int)
              Math.min(
                  Math.min(
                      batchSize / bitWidthAllFixedFields,
                      footer.getBlocks().get(0).getColumns().get(0).getValueCount()),
                  65535);
    } else {
      recordsPerBatch = DEFAULT_RECORDS_TO_READ_IF_NOT_FIXED_WIDTH;
    }

    try {
      ValueVector vector;
      SchemaElement schemaElement;
      final ArrayList<VarLengthColumn<? extends ValueVector>> varLengthColumns = new ArrayList<>();
      // initialize all of the column read status objects
      boolean fieldFixedLength;
      // the column chunk meta-data is not guaranteed to be in the same order as the columns in the
      // schema
      // a map is constructed for fast access to the correct columnChunkMetadata to correspond
      // to an element in the schema
      Map<String, Integer> columnChunkMetadataPositionsInList = new HashMap<>();
      BlockMetaData rowGroupMetadata = footer.getBlocks().get(rowGroupIndex);

      int colChunkIndex = 0;
      for (ColumnChunkMetaData colChunk : rowGroupMetadata.getColumns()) {
        columnChunkMetadataPositionsInList.put(
            Arrays.toString(colChunk.getPath().toArray()), colChunkIndex);
        colChunkIndex++;
      }
      for (int i = 0; i < columns.size(); ++i) {
        column = columns.get(i);
        columnChunkMetaData =
            rowGroupMetadata
                .getColumns()
                .get(columnChunkMetadataPositionsInList.get(Arrays.toString(column.getPath())));
        schemaElement = schemaElements.get(column.getPath()[0]);
        MajorType type =
            ParquetToDrillTypeConverter.toMajorType(
                column.getType(),
                schemaElement.getType_length(),
                getDataMode(column),
                schemaElement,
                fragmentContext.getOptions());
        field = MaterializedField.create(toFieldName(column.getPath()), type);
        // the field was not requested to be read
        if (!fieldSelected(field)) {
          continue;
        }

        fieldFixedLength = column.getType() != PrimitiveType.PrimitiveTypeName.BINARY;
        vector =
            output.addField(
                field,
                (Class<? extends ValueVector>)
                    TypeHelper.getValueVectorClass(type.getMinorType(), type.getMode()));
        if (column.getType() != PrimitiveType.PrimitiveTypeName.BINARY) {
          if (column.getMaxRepetitionLevel() > 0) {
            final RepeatedValueVector repeatedVector = RepeatedValueVector.class.cast(vector);
            ColumnReader<?> dataReader =
                ColumnReaderFactory.createFixedColumnReader(
                    this,
                    fieldFixedLength,
                    column,
                    columnChunkMetaData,
                    recordsPerBatch,
                    repeatedVector.getDataVector(),
                    schemaElement);
            varLengthColumns.add(
                new FixedWidthRepeatedReader(
                    this,
                    dataReader,
                    getTypeLengthInBits(column.getType()),
                    -1,
                    column,
                    columnChunkMetaData,
                    false,
                    repeatedVector,
                    schemaElement));
          } else {
            columnStatuses.add(
                ColumnReaderFactory.createFixedColumnReader(
                    this,
                    fieldFixedLength,
                    column,
                    columnChunkMetaData,
                    recordsPerBatch,
                    vector,
                    schemaElement));
          }
        } else {
          // create a reader and add it to the appropriate list
          varLengthColumns.add(
              ColumnReaderFactory.getReader(
                  this, -1, column, columnChunkMetaData, false, vector, schemaElement));
        }
      }
      varLengthReader = new VarLenBinaryReader(this, varLengthColumns);

      if (!isStarQuery()) {
        List<SchemaPath> projectedColumns = Lists.newArrayList(getColumns());
        SchemaPath col;
        for (int i = 0; i < columnsFound.length; i++) {
          col = projectedColumns.get(i);
          assert col != null;
          if (!columnsFound[i] && !col.equals(STAR_COLUMN)) {
            nullFilledVectors.add(
                (NullableIntVector)
                    output.addField(
                        MaterializedField.create(
                            col.getAsUnescapedPath(), Types.optional(TypeProtos.MinorType.INT)),
                        (Class<? extends ValueVector>)
                            TypeHelper.getValueVectorClass(
                                TypeProtos.MinorType.INT, DataMode.OPTIONAL)));
          }
        }
      }
    } catch (Exception e) {
      handleAndRaise("Failure in setting up reader", e);
    }
  }