You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@impala.apache.org by ta...@apache.org on 2018/02/09 16:59:38 UTC

[19/21] impala git commit: Revert "IMPALA-6219: Use AES-GCM for spill-to-disk encryption"

Revert "IMPALA-6219: Use AES-GCM for spill-to-disk encryption"

This reverts commit 9b68645f9eb9e08899fda860e0946cc05f205479.

Change-Id: Ia06f061a4ecedd1df0d359fe06fe84618b5e07fb
Reviewed-on: http://gerrit.cloudera.org:8080/9226
Reviewed-by: Sailesh Mukil <sa...@cloudera.com>
Tested-by: Impala Public Jenkins


Project: http://git-wip-us.apache.org/repos/asf/impala/repo
Commit: http://git-wip-us.apache.org/repos/asf/impala/commit/31f38687
Tree: http://git-wip-us.apache.org/repos/asf/impala/tree/31f38687
Diff: http://git-wip-us.apache.org/repos/asf/impala/diff/31f38687

Branch: refs/heads/2.x
Commit: 31f38687fa5ab010247e88c2055e00cbeb70712d
Parents: 2037222
Author: Alex Behm <al...@cloudera.com>
Authored: Tue Feb 6 11:50:51 2018 -0800
Committer: Impala Public Jenkins <im...@gerrit.cloudera.org>
Committed: Thu Feb 8 07:01:53 2018 +0000

----------------------------------------------------------------------
 be/src/runtime/tmp-file-mgr.cc   | 15 ++----
 be/src/util/cpu-info.cc          | 13 +++--
 be/src/util/cpu-info.h           | 13 +++--
 be/src/util/openssl-util-test.cc | 95 +++++++++++-----------------------
 be/src/util/openssl-util.cc      | 96 +++--------------------------------
 be/src/util/openssl-util.h       | 70 ++++++++-----------------
 6 files changed, 74 insertions(+), 228 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/impala/blob/31f38687/be/src/runtime/tmp-file-mgr.cc
----------------------------------------------------------------------
diff --git a/be/src/runtime/tmp-file-mgr.cc b/be/src/runtime/tmp-file-mgr.cc
index 3807670..d35d302 100644
--- a/be/src/runtime/tmp-file-mgr.cc
+++ b/be/src/runtime/tmp-file-mgr.cc
@@ -612,26 +612,19 @@ void TmpFileMgr::WriteHandle::WaitForWrite() {
 Status TmpFileMgr::WriteHandle::EncryptAndHash(MemRange buffer) {
   DCHECK(FLAGS_disk_spill_encryption);
   SCOPED_TIMER(encryption_timer_);
-  // Since we're using GCM/CTR/CFB mode, we must take care not to reuse a
+  // Since we're using AES-CTR/AES-CFB mode, we must take care not to reuse a
   // key/IV pair. Regenerate a new key and IV for every data buffer we write.
   key_.InitializeRandom();
   RETURN_IF_ERROR(key_.Encrypt(buffer.data(), buffer.len(), buffer.data()));
-
-  if (!key_.IsGcmMode()) {
-    hash_.Compute(buffer.data(), buffer.len());
-  }
+  hash_.Compute(buffer.data(), buffer.len());
   return Status::OK();
 }
 
 Status TmpFileMgr::WriteHandle::CheckHashAndDecrypt(MemRange buffer) {
   DCHECK(FLAGS_disk_spill_encryption);
   SCOPED_TIMER(encryption_timer_);
-
-  // GCM mode will verify the integrity by itself
-  if (!key_.IsGcmMode()) {
-    if (!hash_.Verify(buffer.data(), buffer.len())) {
-      return Status("Block verification failure");
-    }
+  if (!hash_.Verify(buffer.data(), buffer.len())) {
+    return Status("Block verification failure");
   }
   return key_.Decrypt(buffer.data(), buffer.len(), buffer.data());
 }

http://git-wip-us.apache.org/repos/asf/impala/blob/31f38687/be/src/util/cpu-info.cc
----------------------------------------------------------------------
diff --git a/be/src/util/cpu-info.cc b/be/src/util/cpu-info.cc
index 1e3fcde..a32571e 100644
--- a/be/src/util/cpu-info.cc
+++ b/be/src/util/cpu-info.cc
@@ -85,13 +85,12 @@ static struct {
   int64_t flag;
 } flag_mappings[] =
 {
-  { "ssse3",     CpuInfo::SSSE3 },
-  { "sse4_1",    CpuInfo::SSE4_1 },
-  { "sse4_2",    CpuInfo::SSE4_2 },
-  { "popcnt",    CpuInfo::POPCNT },
-  { "avx",       CpuInfo::AVX },
-  { "avx2",      CpuInfo::AVX2 },
-  { "pclmuldqd", CpuInfo::PCLMULQDQ }
+  { "ssse3",  CpuInfo::SSSE3 },
+  { "sse4_1", CpuInfo::SSE4_1 },
+  { "sse4_2", CpuInfo::SSE4_2 },
+  { "popcnt", CpuInfo::POPCNT },
+  { "avx",    CpuInfo::AVX },
+  { "avx2",   CpuInfo::AVX2 },
 };
 static const long num_flags = sizeof(flag_mappings) / sizeof(flag_mappings[0]);
 

http://git-wip-us.apache.org/repos/asf/impala/blob/31f38687/be/src/util/cpu-info.h
----------------------------------------------------------------------
diff --git a/be/src/util/cpu-info.h b/be/src/util/cpu-info.h
index e60babc..38d6782 100644
--- a/be/src/util/cpu-info.h
+++ b/be/src/util/cpu-info.h
@@ -34,13 +34,12 @@ namespace impala {
 /// /sys/devices)
 class CpuInfo {
  public:
-  static const int64_t SSSE3     = (1 << 1);
-  static const int64_t SSE4_1    = (1 << 2);
-  static const int64_t SSE4_2    = (1 << 3);
-  static const int64_t POPCNT    = (1 << 4);
-  static const int64_t AVX       = (1 << 5);
-  static const int64_t AVX2      = (1 << 6);
-  static const int64_t PCLMULQDQ = (1 << 7);
+  static const int64_t SSSE3   = (1 << 1);
+  static const int64_t SSE4_1  = (1 << 2);
+  static const int64_t SSE4_2  = (1 << 3);
+  static const int64_t POPCNT  = (1 << 4);
+  static const int64_t AVX     = (1 << 5);
+  static const int64_t AVX2    = (1 << 6);
 
   /// Cache enums for L1 (data), L2 and L3
   enum CacheLevel {

http://git-wip-us.apache.org/repos/asf/impala/blob/31f38687/be/src/util/openssl-util-test.cc
----------------------------------------------------------------------
diff --git a/be/src/util/openssl-util-test.cc b/be/src/util/openssl-util-test.cc
index 76f65a5..8d98b0d 100644
--- a/be/src/util/openssl-util-test.cc
+++ b/be/src/util/openssl-util-test.cc
@@ -44,41 +44,6 @@ class OpenSSLUtilTest : public ::testing::Test {
     }
   }
 
-  /// Fill arbitrary-length buffer with random bytes
-  void GenerateRandomBytes(uint8_t* data, int64_t len) {
-    DCHECK_GE(len, 0);
-    for (int64_t i = 0; i < len; i++) {
-      data[i] = uniform_int_distribution<uint8_t>(0, UINT8_MAX)(rng_);
-    }
-  }
-
-  void TestEncryptionDecryption(const int64_t buffer_size) {
-    vector<uint8_t> original(buffer_size);
-    vector<uint8_t> scratch(buffer_size); // Scratch buffer for in-place encryption.
-    if (buffer_size % 8 == 0) {
-      GenerateRandomData(original.data(), buffer_size);
-    } else {
-      GenerateRandomBytes(original.data(), buffer_size);
-    }
-
-    // Check all the modes
-    AES_CIPHER_MODE modes[] = {AES_256_GCM, AES_256_CTR, AES_256_CFB};
-    for (auto m : modes) {
-      memcpy(scratch.data(), original.data(), buffer_size);
-
-      EncryptionKey key;
-      key.InitializeRandom();
-      key.SetCipherMode(m);
-
-      ASSERT_OK(key.Encrypt(scratch.data(), buffer_size, scratch.data()));
-      // Check that encryption did something
-      ASSERT_NE(0, memcmp(original.data(), scratch.data(), buffer_size));
-      ASSERT_OK(key.Decrypt(scratch.data(), buffer_size, scratch.data()));
-      // Check that we get the original data back.
-      ASSERT_EQ(0, memcmp(original.data(), scratch.data(), buffer_size));
-    }
-  }
-
   mt19937_64 rng_;
 };
 
@@ -92,7 +57,7 @@ TEST_F(OpenSSLUtilTest, Encryption) {
   GenerateRandomData(original.data(), buffer_size);
 
   // Check both CTR & CFB
-  AES_CIPHER_MODE modes[] = {AES_256_GCM, AES_256_CTR, AES_256_CFB};
+  AES_CIPHER_MODE modes[] = {AES_256_CTR, AES_256_CFB};
   for (auto m : modes) {
     // Iterate multiple times to ensure that key regeneration works correctly.
     EncryptionKey key;
@@ -120,42 +85,44 @@ TEST_F(OpenSSLUtilTest, Encryption) {
 /// Test that encryption and decryption work in-place.
 TEST_F(OpenSSLUtilTest, EncryptInPlace) {
   const int buffer_size = 1024 * 1024;
-  TestEncryptionDecryption(buffer_size);
+  vector<uint8_t> original(buffer_size);
+  vector<uint8_t> scratch(buffer_size); // Scratch buffer for in-place encryption.
+
+  EncryptionKey key;
+  // Check both CTR & CFB
+  AES_CIPHER_MODE modes[] = {AES_256_CTR, AES_256_CFB};
+  for (auto m : modes) {
+    GenerateRandomData(original.data(), buffer_size);
+    memcpy(scratch.data(), original.data(), buffer_size);
+
+    key.InitializeRandom();
+    key.SetCipherMode(m);
+
+    ASSERT_OK(key.Encrypt(scratch.data(), buffer_size, scratch.data()));
+    // Check that encryption did something
+    ASSERT_NE(0, memcmp(original.data(), scratch.data(), buffer_size));
+    ASSERT_OK(key.Decrypt(scratch.data(), buffer_size, scratch.data()));
+    // Check that we get the original data back.
+    ASSERT_EQ(0, memcmp(original.data(), scratch.data(), buffer_size));
+  }
 }
 
 /// Test that encryption works with buffer lengths that don't fit in a 32-bit integer.
 TEST_F(OpenSSLUtilTest, EncryptInPlaceHugeBuffer) {
   const int64_t buffer_size = 3 * 1024L * 1024L * 1024L;
-  TestEncryptionDecryption(buffer_size);
-}
-
-/// Test that encryption works with arbitrary-length buffer
-TEST_F(OpenSSLUtilTest, EncryptArbitraryLength) {
-  std::uniform_int_distribution<uint64_t> dis(0, 1024 * 1024);
-  const int buffer_size = dis(rng_);
-  TestEncryptionDecryption(buffer_size);
-}
-
-/// Test integrity in GCM mode
-TEST_F(OpenSSLUtilTest, GcmIntegrity) {
-  const int buffer_size = 1024 * 1024;
-  vector<uint8_t> buffer(buffer_size);
+  vector<uint8_t> original(buffer_size);
+  vector<uint8_t> scratch(buffer_size); // Scratch buffer for in-place encryption.
+  GenerateRandomData(original.data(), buffer_size);
+  memcpy(scratch.data(), original.data(), buffer_size);
 
   EncryptionKey key;
   key.InitializeRandom();
-  key.SetCipherMode(AES_256_GCM);
-
-  // Even it has been set as GCM mode, it may fall back to other modes.
-  // Check if GCM mode is supported at runtime.
-  if (key.IsGcmMode()) {
-    GenerateRandomData(buffer.data(), buffer_size);
-    ASSERT_OK(key.Encrypt(buffer.data(), buffer_size, buffer.data()));
-
-    // tamper the data
-    ++buffer[0];
-    Status s = key.Decrypt(buffer.data(), buffer_size, buffer.data());
-    EXPECT_STR_CONTAINS(s.GetDetail(), "EVP_DecryptFinal");
-  }
+  ASSERT_OK(key.Encrypt(scratch.data(), buffer_size, scratch.data()));
+  // Check that encryption did something
+  ASSERT_NE(0, memcmp(original.data(), scratch.data(), buffer_size));
+  ASSERT_OK(key.Decrypt(scratch.data(), buffer_size, scratch.data()));
+  // Check that we get the original data back.
+  ASSERT_EQ(0, memcmp(original.data(), scratch.data(), buffer_size));
 }
 
 /// Test basic integrity hash functionality.

http://git-wip-us.apache.org/repos/asf/impala/blob/31f38687/be/src/util/openssl-util.cc
----------------------------------------------------------------------
diff --git a/be/src/util/openssl-util.cc b/be/src/util/openssl-util.cc
index ffb47eb..69dc676 100644
--- a/be/src/util/openssl-util.cc
+++ b/be/src/util/openssl-util.cc
@@ -20,7 +20,6 @@
 #include <limits.h>
 #include <sstream>
 
-#include <glog/logging.h>
 #include <openssl/err.h>
 #include <openssl/evp.h>
 #include <openssl/rand.h>
@@ -31,7 +30,6 @@
 #include "gutil/strings/substitute.h"
 
 #include "common/names.h"
-#include "cpu-info.h"
 
 DECLARE_string(ssl_client_ca_certificate);
 DECLARE_string(ssl_server_certificate);
@@ -109,20 +107,19 @@ void EncryptionKey::InitializeRandom() {
   }
   RAND_bytes(key_, sizeof(key_));
   RAND_bytes(iv_, sizeof(iv_));
-  memset(gcm_tag_, 0, sizeof(gcm_tag_));
   initialized_ = true;
 }
 
-Status EncryptionKey::Encrypt(const uint8_t* data, int64_t len, uint8_t* out) {
+Status EncryptionKey::Encrypt(const uint8_t* data, int64_t len, uint8_t* out) const {
   return EncryptInternal(true, data, len, out);
 }
 
-Status EncryptionKey::Decrypt(const uint8_t* data, int64_t len, uint8_t* out) {
+Status EncryptionKey::Decrypt(const uint8_t* data, int64_t len, uint8_t* out) const {
   return EncryptInternal(false, data, len, out);
 }
 
 Status EncryptionKey::EncryptInternal(
-    bool encrypt, const uint8_t* data, int64_t len, uint8_t* out) {
+    bool encrypt, const uint8_t* data, int64_t len, uint8_t* out) const {
   DCHECK(initialized_);
   DCHECK_GE(len, 0);
   // Create and initialize the context for encryption
@@ -130,10 +127,6 @@ Status EncryptionKey::EncryptInternal(
   EVP_CIPHER_CTX_init(&ctx);
   EVP_CIPHER_CTX_set_padding(&ctx, 0);
 
-  if (IsGcmMode()) {
-    EVP_CIPHER_CTX_ctrl(&ctx, EVP_CTRL_GCM_SET_IVLEN, AES_BLOCK_SIZE, NULL);
-  }
-
   // Start encryption/decryption.  We use a 256-bit AES key, and the cipher block mode
   // is either CTR or CFB(stream cipher), both of which support arbitrary length
   // ciphertexts - it doesn't have to be a multiple of 16 bytes. Additionally, CTR
@@ -164,11 +157,6 @@ Status EncryptionKey::EncryptInternal(
     offset += in_len;
   }
 
-  if (IsGcmMode() && !encrypt) {
-    // Set expected tag value
-    EVP_CIPHER_CTX_ctrl(&ctx, EVP_CTRL_GCM_SET_TAG, AES_BLOCK_SIZE, gcm_tag_);
-  }
-
   // Finalize encryption or decryption.
   int final_out_len;
   success = encrypt ? EVP_EncryptFinal_ex(&ctx, out + offset, &final_out_len) :
@@ -176,93 +164,21 @@ Status EncryptionKey::EncryptInternal(
   if (success != 1) {
     return OpenSSLErr(encrypt ? "EVP_EncryptFinal" : "EVP_DecryptFinal");
   }
-
-  if (IsGcmMode() && encrypt) {
-    EVP_CIPHER_CTX_ctrl(&ctx, EVP_CTRL_GCM_GET_TAG, AES_BLOCK_SIZE, gcm_tag_);
-  }
-
-  // Again safe due to GCM/CTR/CFB with no padding
+  // Again safe due to CTR/CFB with no padding
   DCHECK_EQ(final_out_len, 0);
   return Status::OK();
 }
 
-/// OpenSSL 1.0.1d
-#define OPENSSL_VERSION_1_0_1D 0x1000104fL
-
-/// If not defined at compile time, define them manually
-/// see: openssl/evp.h
-#ifndef EVP_CIPH_GCM_MODE
-#define EVP_CTRL_GCM_SET_IVLEN 0x9
-#define EVP_CTRL_GCM_GET_TAG 0x10
-#define EVP_CTRL_GCM_SET_TAG 0x11
-#endif
-
 extern "C" {
 ATTRIBUTE_WEAK
 const EVP_CIPHER* EVP_aes_256_ctr();
-
-ATTRIBUTE_WEAK
-const EVP_CIPHER* EVP_aes_256_gcm();
 }
 
 const EVP_CIPHER* EncryptionKey::GetCipher() const {
   // use weak symbol to avoid compiling error on OpenSSL 1.0.0 environment
-  if (mode_ == AES_256_CTR) return EVP_aes_256_ctr();
-  if (mode_ == AES_256_GCM) return EVP_aes_256_gcm();
+  if (mode_ == AES_256_CTR && EVP_aes_256_ctr) return EVP_aes_256_ctr();
 
+  // otherwise, fallback to CFB mode
   return EVP_aes_256_cfb();
 }
-
-void EncryptionKey::SetCipherMode(AES_CIPHER_MODE m) {
-  mode_ = m;
-
-  if (!IsModeSupported(m)) {
-    mode_ = GetSupportedDefaultMode();
-    LOG(WARNING) << Substitute("$0 is not supported, fall back to $1.",
-        ModeToString(m), ModeToString(mode_));
-  }
-}
-
-bool EncryptionKey::IsModeSupported(AES_CIPHER_MODE m) const {
-  switch (m) {
-    case AES_256_GCM:
-      // It becomes a bit tricky for GCM mode, because GCM mode is enabled since
-      // OpenSSL 1.0.1, but the tag validation only works since 1.0.1d. We have
-      // to make sure that OpenSSL version >= 1.0.1d for GCM. So we need
-      // SSLeay(). Note that SSLeay() may return the compiling version on
-      // certain platforms if it was built against an older version(see:
-      // IMPALA-6418). In this case, it will return false, and EncryptionKey
-      // will try to fall back to CTR mode, so it is not ideal but is OK to use
-      // SSLeay() for GCM mode here since in the worst case, we will be using
-      // AES_256_CTR in a system that supports AES_256_GCM.
-      return (CpuInfo::IsSupported(CpuInfo::PCLMULQDQ)
-          && SSLeay() >= OPENSSL_VERSION_1_0_1D && EVP_aes_256_gcm);
-
-    case AES_256_CTR:
-      // If TLS1.2 is supported, then we're on a verison of OpenSSL that
-      // supports AES-256-CTR.
-      return (MaxSupportedTlsVersion() >= TLS1_2_VERSION && EVP_aes_256_ctr);
-
-    case AES_256_CFB:
-      return true;
-
-    default:
-      return false;
-  }
-}
-
-AES_CIPHER_MODE EncryptionKey::GetSupportedDefaultMode() const {
-  if (IsModeSupported(AES_256_GCM)) return AES_256_GCM;
-  if (IsModeSupported(AES_256_CTR)) return AES_256_CTR;
-  return AES_256_CFB;
-}
-
-const string EncryptionKey::ModeToString(AES_CIPHER_MODE m) const {
-  switch(m) {
-    case AES_256_GCM: return "AES-GCM";
-    case AES_256_CTR: return "AES-CTR";
-    case AES_256_CFB: return "AES-CFB";
-  }
-  return "Unknown mode";
-}
 }

http://git-wip-us.apache.org/repos/asf/impala/blob/31f38687/be/src/util/openssl-util.h
----------------------------------------------------------------------
diff --git a/be/src/util/openssl-util.h b/be/src/util/openssl-util.h
index ef53425..7b1b28e 100644
--- a/be/src/util/openssl-util.h
+++ b/be/src/util/openssl-util.h
@@ -60,9 +60,9 @@ bool IsExternalTlsConfigured();
 void SeedOpenSSLRNG();
 
 enum AES_CIPHER_MODE {
-  AES_256_CFB,
   AES_256_CTR,
-  AES_256_GCM
+  AES_256_CFB,
+  AES_256_GCM // not supported now.
 };
 
 /// The hash of a data buffer used for checking integrity. A SHA256 hash is used
@@ -83,56 +83,43 @@ class IntegrityHash {
 /// The key and initialization vector (IV) required to encrypt and decrypt a buffer of
 /// data. This should be regenerated for each buffer of data.
 ///
-/// We use AES with a 256-bit key and GCM/CTR/CFB cipher block mode, which gives us a
-/// stream cipher that can support arbitrary-length ciphertexts. The mode is chosen
-/// depends on the OpenSSL version & the hardware support at runtime. The IV is used as
+/// We use AES with a 256-bit key and CTR/CFB cipher block mode, which gives us a stream
+/// cipher that can support arbitrary-length ciphertexts. If OpenSSL version at runtime
+/// is 1.0.1 or above, CTR mode is used, otherwise CFB mode is used. The IV is used as
 /// an input to the cipher as the "block to supply before the first block of plaintext".
 /// This is required because all ciphers (except the weak ECB) are built such that each
 /// block depends on the output from the previous block. Since the first block doesn't
 /// have a previous block, we supply this IV. Think of it  as starting off the chain of
 /// encryption.
-///
-/// Notes for GCM:
-/// (1) GCM mode was supported since OpenSSL 1.0.1, however the tag verification
-/// in decryption was only supported since OpenSSL 1.0.1d.
-/// (2) The plaintext and the Additional Authenticated Data(AAD) are the two
-/// categories of data that GCM protects. GCM protects the authenticity of the
-/// plaintext and the AAD, and GCM also protects the confidentiality of the
-/// plaintext. The AAD itself is not required or won't change the security.
-/// In our case(Spill to Disk), we just ignore the AAD.
-
 class EncryptionKey {
  public:
-  EncryptionKey() : initialized_(false) { mode_ = GetSupportedDefaultMode(); }
-
-  /// Initializes a key for temporary use with randomly generated data, and clears the
-  /// tag for GCM mode. Reinitializes with new random values if the key was already
-  /// initialized. We use AES-GCM/AES-CTR/AES-CFB mode so key/IV pairs should not be
-  /// reused. This function automatically reseeds the RNG periodically, so callers do
-  /// not need to do it.
+  EncryptionKey() : initialized_(false) {
+    // If TLS1.2 is supported, then we're on a verison of OpenSSL that supports
+    // AES-256-CTR.
+    mode_ = MaxSupportedTlsVersion() < TLS1_2_VERSION ? AES_256_CFB : AES_256_CTR;
+  }
+
+  /// Initialize a key for temporary use with randomly generated data. Reinitializes with
+  /// new random values if the key was already initialized. We use AES-CTR/AES-CFB mode
+  /// so key/IV pairs should not be reused. This function automatically reseeds the RNG
+  /// periodically, so callers do not need to do it.
   void InitializeRandom();
 
   /// Encrypts a buffer of input data 'data' of length 'len' into an output buffer 'out'.
   /// Exactly 'len' bytes will be written to 'out'. This key must be initialized before
   /// calling. Operates in-place if 'in' == 'out', otherwise the buffers must not overlap.
-  /// For GCM mode, the hash tag will be kept inside(gcm_tag_ variable).
-  Status Encrypt(const uint8_t* data, int64_t len, uint8_t* out) WARN_UNUSED_RESULT;
+  Status Encrypt(const uint8_t* data, int64_t len, uint8_t* out) const WARN_UNUSED_RESULT;
 
   /// Decrypts a buffer of input data 'data' of length 'len' that was encrypted with this
   /// key into an output buffer 'out'. Exactly 'len' bytes will be written to 'out'.
   /// This key must be initialized before calling. Operates in-place if 'in' == 'out',
-  /// otherwise the buffers must not overlap. For GCM mode, the hash tag, which is
-  /// computed during encryption, will be used for intgerity verification.
-  Status Decrypt(const uint8_t* data, int64_t len, uint8_t* out) WARN_UNUSED_RESULT;
+  /// otherwise the buffers must not overlap.
+  Status Decrypt(const uint8_t* data, int64_t len, uint8_t* out) const WARN_UNUSED_RESULT;
 
   /// Specify a cipher mode. Currently used only for testing but maybe in future we
   /// can provide a configuration option for the end user who can choose a preferred
   /// mode(GCM, CTR, CFB...) based on their software/hardware environment.
-  /// If not supported, fall back to the supported mode at runtime.
-  void SetCipherMode(AES_CIPHER_MODE m);
-
-  /// If is GCM mode at runtime
-  bool IsGcmMode() const { return mode_ == AES_256_GCM; }
+  void SetCipherMode(AES_CIPHER_MODE m) { mode_ = m; }
 
  private:
   /// Helper method that encrypts/decrypts if 'encrypt' is true/false respectively.
@@ -141,25 +128,13 @@ class EncryptionKey {
   /// This key must be initialized before calling. Operates in-place if 'in' == 'out',
   /// otherwise the buffers must not overlap.
   Status EncryptInternal(bool encrypt, const uint8_t* data, int64_t len,
-      uint8_t* out) WARN_UNUSED_RESULT;
-
-  /// Check if mode m is supported at runtime
-  bool IsModeSupported(AES_CIPHER_MODE m) const;
-
-  /// Returns the a default mode which is supported at runtime. If GCM mode
-  /// is supported, return AES_256_GCM as the default. If GCM is not supported,
-  /// but CTR is still supported, return AES_256_CTR. When both GCM and
-  /// CTR modes are not supported, return AES_256_CFB.
-  AES_CIPHER_MODE GetSupportedDefaultMode() const;
-
-  /// Converts mode type to string.
-  const string ModeToString(AES_CIPHER_MODE m) const;
+      uint8_t* out) const WARN_UNUSED_RESULT;
 
   /// Track whether this key has been initialized, to avoid accidentally using
   /// uninitialized keys.
   bool initialized_;
 
-  /// Returns a EVP_CIPHER according to cipher mode at runtime
+  /// return a EVP_CIPHER according to cipher mode at runtime
   const EVP_CIPHER* GetCipher() const;
 
   /// An AES 256-bit key.
@@ -168,9 +143,6 @@ class EncryptionKey {
   /// An initialization vector to feed as the first block to AES.
   uint8_t iv_[AES_BLOCK_SIZE];
 
-  /// Tag for GCM mode
-  uint8_t gcm_tag_[AES_BLOCK_SIZE];
-
   /// Cipher Mode
   AES_CIPHER_MODE mode_;
 };