private void addRowGroup()

in parquet-hadoop/src/main/java/org/apache/parquet/format/converter/ParquetMetadataConverter.java [549:666]


  private void addRowGroup(
      ParquetMetadata parquetMetadata,
      List<RowGroup> rowGroups,
      BlockMetaData block,
      InternalFileEncryptor fileEncryptor) {

    // rowGroup.total_byte_size = ;
    List<ColumnChunkMetaData> columns = block.getColumns();
    List<ColumnChunk> parquetColumns = new ArrayList<ColumnChunk>();
    int rowGroupOrdinal = rowGroups.size();
    int columnOrdinal = -1;
    ByteArrayOutputStream tempOutStream = null;
    for (ColumnChunkMetaData columnMetaData : columns) {
      // There is no ColumnMetaData written after the chunk data, so set the ColumnChunk
      // file_offset to 0
      ColumnChunk columnChunk = new ColumnChunk(0);
      columnChunk.file_path = block.getPath(); // they are in the same file for now
      InternalColumnEncryptionSetup columnSetup = null;
      boolean writeCryptoMetadata = false;
      boolean encryptMetaData = false;
      ColumnPath path = columnMetaData.getPath();
      if (null != fileEncryptor) {
        columnOrdinal++;
        columnSetup = fileEncryptor.getColumnSetup(path, false, columnOrdinal);
        writeCryptoMetadata = columnSetup.isEncrypted();
        encryptMetaData = fileEncryptor.encryptColumnMetaData(columnSetup);
      }
      ColumnMetaData metaData = new ColumnMetaData(
          getType(columnMetaData.getType()),
          toFormatEncodings(columnMetaData.getEncodings()),
          columnMetaData.getPath().toList(),
          toFormatCodec(columnMetaData.getCodec()),
          columnMetaData.getValueCount(),
          columnMetaData.getTotalUncompressedSize(),
          columnMetaData.getTotalSize(),
          columnMetaData.getFirstDataPageOffset());
      if ((columnMetaData.getEncodingStats() != null
              && columnMetaData.getEncodingStats().hasDictionaryPages())
          || columnMetaData.hasDictionaryPage()) {
        metaData.setDictionary_page_offset(columnMetaData.getDictionaryPageOffset());
      }
      long bloomFilterOffset = columnMetaData.getBloomFilterOffset();
      if (bloomFilterOffset >= 0) {
        metaData.setBloom_filter_offset(bloomFilterOffset);
      }
      int bloomFilterLength = columnMetaData.getBloomFilterLength();
      if (bloomFilterLength >= 0) {
        metaData.setBloom_filter_length(bloomFilterLength);
      }
      if (columnMetaData.getStatistics() != null
          && !columnMetaData.getStatistics().isEmpty()) {
        metaData.setStatistics(
            toParquetStatistics(columnMetaData.getStatistics(), this.statisticsTruncateLength));
      }
      if (columnMetaData.getEncodingStats() != null) {
        metaData.setEncoding_stats(convertEncodingStats(columnMetaData.getEncodingStats()));
      }
      if (columnMetaData.getSizeStatistics() != null
          && columnMetaData.getSizeStatistics().isValid()) {
        metaData.setSize_statistics(toParquetSizeStatistics(columnMetaData.getSizeStatistics()));
      }

      if (!encryptMetaData) {
        columnChunk.setMeta_data(metaData);
      } else {
        // Serialize and encrypt ColumnMetadata separately
        byte[] columnMetaDataAAD = AesCipher.createModuleAAD(
            fileEncryptor.getFileAAD(),
            ModuleType.ColumnMetaData,
            rowGroupOrdinal,
            columnSetup.getOrdinal(),
            -1);
        if (null == tempOutStream) {
          tempOutStream = new ByteArrayOutputStream();
        } else {
          tempOutStream.reset();
        }
        try {
          writeColumnMetaData(metaData, tempOutStream, columnSetup.getMetaDataEncryptor(), columnMetaDataAAD);
        } catch (IOException e) {
          throw new ParquetCryptoRuntimeException(
              "Failed to serialize and encrypt ColumnMetadata for " + columnMetaData.getPath(), e);
        }
        columnChunk.setEncrypted_column_metadata(tempOutStream.toByteArray());
        // Keep redacted metadata version
        if (!fileEncryptor.isFooterEncrypted()) {
          ColumnMetaData metaDataRedacted = metaData.deepCopy();
          if (metaDataRedacted.isSetStatistics()) metaDataRedacted.unsetStatistics();
          if (metaDataRedacted.isSetEncoding_stats()) metaDataRedacted.unsetEncoding_stats();
          columnChunk.setMeta_data(metaDataRedacted);
        }
      }
      if (writeCryptoMetadata) {
        columnChunk.setCrypto_metadata(columnSetup.getColumnCryptoMetaData());
      }

      //      columnChunk.meta_data.index_page_offset = ;
      //      columnChunk.meta_data.key_value_metadata = ; // nothing yet

      IndexReference columnIndexRef = columnMetaData.getColumnIndexReference();
      if (columnIndexRef != null) {
        columnChunk.setColumn_index_offset(columnIndexRef.getOffset());
        columnChunk.setColumn_index_length(columnIndexRef.getLength());
      }
      IndexReference offsetIndexRef = columnMetaData.getOffsetIndexReference();
      if (offsetIndexRef != null) {
        columnChunk.setOffset_index_offset(offsetIndexRef.getOffset());
        columnChunk.setOffset_index_length(offsetIndexRef.getLength());
      }

      parquetColumns.add(columnChunk);
    }
    RowGroup rowGroup = new RowGroup(parquetColumns, block.getTotalByteSize(), block.getRowCount());
    rowGroup.setFile_offset(block.getStartingPos());
    rowGroup.setTotal_compressed_size(block.getCompressedSize());
    rowGroup.setOrdinal((short) rowGroupOrdinal);
    rowGroups.add(rowGroup);
  }