Uploaded image for project: 'Hadoop Common'
  1. Hadoop Common
  2. HADOOP-11343

Overflow is not properly handled in caclulating final iv for AES CTR

    XMLWordPrintableJSON

Details

    Description

      In the AesCtrCryptoCodec calculateIV, as the init IV is a random generated 16 bytes,

      final byte[] iv = new byte[cc.getCipherSuite().getAlgorithmBlockSize()];
      cc.generateSecureRandom(iv);

      Then the following calculation of iv and counter on 8 bytes (64bit) space would easily cause overflow and this overflow gets lost. The result would be the 128 bit data block was encrypted with a wrong counter and cannot be decrypted by standard aes-ctr.

      /**
         * The IV is produced by adding the initial IV to the counter. IV length 
         * should be the same as {@link #AES_BLOCK_SIZE}
         */
        @Override
        public void calculateIV(byte[] initIV, long counter, byte[] IV) {
          Preconditions.checkArgument(initIV.length == AES_BLOCK_SIZE);
          Preconditions.checkArgument(IV.length == AES_BLOCK_SIZE);
          
          System.arraycopy(initIV, 0, IV, 0, CTR_OFFSET);
          long l = 0;
          for (int i = 0; i < 8; i++) {
            l = ((l << 8) | (initIV[CTR_OFFSET + i] & 0xff));
          }
          l += counter;
          IV[CTR_OFFSET + 0] = (byte) (l >>> 56);
          IV[CTR_OFFSET + 1] = (byte) (l >>> 48);
          IV[CTR_OFFSET + 2] = (byte) (l >>> 40);
          IV[CTR_OFFSET + 3] = (byte) (l >>> 32);
          IV[CTR_OFFSET + 4] = (byte) (l >>> 24);
          IV[CTR_OFFSET + 5] = (byte) (l >>> 16);
          IV[CTR_OFFSET + 6] = (byte) (l >>> 8);
          IV[CTR_OFFSET + 7] = (byte) (l);
        }
      

      Attachments

        1. HADOOP-11343.patch
          6 kB
          Haifeng Chen
        2. HADOOP-11343.001.patch
          5 kB
          Yi Liu
        3. HADOOP-11343.002.patch
          5 kB
          Yi Liu
        4. HADOOP-11343.003.patch
          5 kB
          Yi Liu

        Activity

          People

            jerrychenhf Haifeng Chen
            jerrychenhf Haifeng Chen
            Votes:
            0 Vote for this issue
            Watchers:
            12 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved: