0001
0002
0003
0004
0005
0006
0007
0008
0009
0010
0011
0012
0013
0014
0015
0016
0017
0018 package org.apache.spark.network.crypto;
0019
0020 import java.nio.ByteBuffer;
0021
0022 import io.netty.buffer.ByteBuf;
0023 import io.netty.buffer.Unpooled;
0024
0025 import org.apache.spark.network.protocol.Encodable;
0026 import org.apache.spark.network.protocol.Encoders;
0027
0028
0029
0030
0031
0032
0033 public class ClientChallenge implements Encodable {
0034
0035 private static final byte TAG_BYTE = (byte) 0xFA;
0036
0037 public final String appId;
0038 public final String kdf;
0039 public final int iterations;
0040 public final String cipher;
0041 public final int keyLength;
0042 public final byte[] nonce;
0043 public final byte[] challenge;
0044
0045 public ClientChallenge(
0046 String appId,
0047 String kdf,
0048 int iterations,
0049 String cipher,
0050 int keyLength,
0051 byte[] nonce,
0052 byte[] challenge) {
0053 this.appId = appId;
0054 this.kdf = kdf;
0055 this.iterations = iterations;
0056 this.cipher = cipher;
0057 this.keyLength = keyLength;
0058 this.nonce = nonce;
0059 this.challenge = challenge;
0060 }
0061
0062 @Override
0063 public int encodedLength() {
0064 return 1 + 4 + 4 +
0065 Encoders.Strings.encodedLength(appId) +
0066 Encoders.Strings.encodedLength(kdf) +
0067 Encoders.Strings.encodedLength(cipher) +
0068 Encoders.ByteArrays.encodedLength(nonce) +
0069 Encoders.ByteArrays.encodedLength(challenge);
0070 }
0071
0072 @Override
0073 public void encode(ByteBuf buf) {
0074 buf.writeByte(TAG_BYTE);
0075 Encoders.Strings.encode(buf, appId);
0076 Encoders.Strings.encode(buf, kdf);
0077 buf.writeInt(iterations);
0078 Encoders.Strings.encode(buf, cipher);
0079 buf.writeInt(keyLength);
0080 Encoders.ByteArrays.encode(buf, nonce);
0081 Encoders.ByteArrays.encode(buf, challenge);
0082 }
0083
0084 public static ClientChallenge decodeMessage(ByteBuffer buffer) {
0085 ByteBuf buf = Unpooled.wrappedBuffer(buffer);
0086
0087 if (buf.readByte() != TAG_BYTE) {
0088 throw new IllegalArgumentException("Expected ClientChallenge, received something else.");
0089 }
0090
0091 return new ClientChallenge(
0092 Encoders.Strings.decode(buf),
0093 Encoders.Strings.decode(buf),
0094 buf.readInt(),
0095 Encoders.Strings.decode(buf),
0096 buf.readInt(),
0097 Encoders.ByteArrays.decode(buf),
0098 Encoders.ByteArrays.decode(buf));
0099 }
0100
0101 }