public FileEncryptionProperties getFileEncryptionProperties()

in parquet-hadoop/src/main/java/org/apache/parquet/crypto/keytools/PropertiesDrivenCryptoFactory.java [85:208]


  public FileEncryptionProperties getFileEncryptionProperties(
      Configuration fileHadoopConfig, Path tempFilePath, WriteContext fileWriteContext)
      throws ParquetCryptoRuntimeException {

    String footerKeyId = fileHadoopConfig.getTrimmed(FOOTER_KEY_PROPERTY_NAME);
    String columnKeysStr = fileHadoopConfig.getTrimmed(COLUMN_KEYS_PROPERTY_NAME);
    String uniformKeyId = fileHadoopConfig.getTrimmed(UNIFORM_KEY_PROPERTY_NAME);
    boolean completeColumnEncryption = fileHadoopConfig.getBoolean(
        COMPLETE_COLUMN_ENCRYPTION_PROPERTY_NAME, COMPLETE_COLUMN_ENCRYPTION_DEFAULT);

    boolean emptyFooterKeyId = stringIsEmpty(footerKeyId);
    boolean emptyColumnKeyIds = stringIsEmpty(columnKeysStr);
    boolean emptyUniformKeyId = stringIsEmpty(uniformKeyId);

    // File shouldn't be encrypted
    if (emptyFooterKeyId && emptyColumnKeyIds && emptyUniformKeyId) {
      LOG.debug("Unencrypted file: {}", tempFilePath);
      return null;
    }

    if (emptyUniformKeyId) {
      // Non-uniform encryption.Must have both footer and column key ids
      if (emptyFooterKeyId) {
        throw new ParquetCryptoRuntimeException("No footer key configured in " + FOOTER_KEY_PROPERTY_NAME);
      }
      if (emptyColumnKeyIds) {
        throw new ParquetCryptoRuntimeException("No column keys configured in " + COLUMN_KEYS_PROPERTY_NAME);
      }
    } else {
      // Uniform encryption. Can't have configuration of footer and column key ids
      if (!emptyFooterKeyId) {
        throw new ParquetCryptoRuntimeException(
            "Uniform encryption. Cant have footer key configured in " + FOOTER_KEY_PROPERTY_NAME);
      }
      if (!emptyColumnKeyIds) {
        throw new ParquetCryptoRuntimeException(
            "Uniform encryption. Cant have column keys configured in " + COLUMN_KEYS_PROPERTY_NAME);
      }
      if (completeColumnEncryption) {
        throw new ParquetCryptoRuntimeException(
            "Complete column encryption cant be applied in uniform encryption mode");
      }

      // Now assign footer key id to uniform key id
      footerKeyId = uniformKeyId;
    }

    FileKeyMaterialStore keyMaterialStore = null;
    boolean keyMaterialInternalStorage = fileHadoopConfig.getBoolean(
        KeyToolkit.KEY_MATERIAL_INTERNAL_PROPERTY_NAME, KeyToolkit.KEY_MATERIAL_INTERNAL_DEFAULT);
    if (!keyMaterialInternalStorage) {
      if (tempFilePath == null) {
        throw new ParquetCryptoRuntimeException("Output file path cannot be null");
      }
      try {
        keyMaterialStore = new HadoopFSKeyMaterialStore(tempFilePath.getFileSystem(fileHadoopConfig));
        keyMaterialStore.initialize(tempFilePath, fileHadoopConfig, false);
      } catch (IOException e) {
        throw new ParquetCryptoRuntimeException("Failed to get key material store", e);
      }
    }

    FileKeyWrapper keyWrapper = new FileKeyWrapper(fileHadoopConfig, keyMaterialStore);

    String algo = fileHadoopConfig.getTrimmed(ENCRYPTION_ALGORITHM_PROPERTY_NAME, ENCRYPTION_ALGORITHM_DEFAULT);
    ParquetCipher cipher;
    try {
      cipher = ParquetCipher.valueOf(algo);
    } catch (IllegalArgumentException e) {
      throw new ParquetCryptoRuntimeException("Wrong encryption algorithm: " + algo);
    }

    int dekLengthBits =
        fileHadoopConfig.getInt(KeyToolkit.DATA_KEY_LENGTH_PROPERTY_NAME, KeyToolkit.DATA_KEY_LENGTH_DEFAULT);

    if (Arrays.binarySearch(ACCEPTABLE_DATA_KEY_LENGTHS, dekLengthBits) < 0) {
      throw new ParquetCryptoRuntimeException("Wrong data key length : " + dekLengthBits);
    }

    int dekLength = dekLengthBits / 8;

    byte[] footerKeyBytes = new byte[dekLength];
    RANDOM.nextBytes(footerKeyBytes);
    byte[] footerKeyMetadata = keyWrapper.getEncryptionKeyMetadata(footerKeyBytes, footerKeyId, true);

    boolean plaintextFooter = fileHadoopConfig.getBoolean(PLAINTEXT_FOOTER_PROPERTY_NAME, PLAINTEXT_FOOTER_DEFAULT);

    FileEncryptionProperties.Builder propertiesBuilder = FileEncryptionProperties.builder(footerKeyBytes)
        .withFooterKeyMetadata(footerKeyMetadata)
        .withAlgorithm(cipher);

    if (emptyUniformKeyId) {
      Map<ColumnPath, ColumnEncryptionProperties> encryptedColumns =
          getColumnEncryptionProperties(dekLength, columnKeysStr, keyWrapper);
      propertiesBuilder = propertiesBuilder.withEncryptedColumns(encryptedColumns);

      if (completeColumnEncryption) {
        propertiesBuilder = propertiesBuilder.withCompleteColumnEncryption();
      }
    }

    if (plaintextFooter) {
      propertiesBuilder = propertiesBuilder.withPlaintextFooter();
    }

    if (null != keyMaterialStore) {
      keyMaterialStore.saveMaterial();
    }

    if (LOG.isDebugEnabled()) {
      LOG.debug(
          "File encryption properties for {} - algo: {}; footer key id: {}; uniform key id: {}; " + ""
              + "plaintext footer: {}; internal key material: {}; encrypted columns: {}",
          tempFilePath,
          cipher,
          footerKeyId,
          uniformKeyId,
          plaintextFooter,
          keyMaterialInternalStorage,
          columnKeysStr);
    }

    return propertiesBuilder.build();
  }