Back to home page

OSCL-LXR

 
 

    


0001 /*
0002  * Licensed to the Apache Software Foundation (ASF) under one or more
0003  * contributor license agreements.  See the NOTICE file distributed with
0004  * this work for additional information regarding copyright ownership.
0005  * The ASF licenses this file to You under the Apache License, Version 2.0
0006  * (the "License"); you may not use this file except in compliance with
0007  * the License.  You may obtain a copy of the License at
0008  *
0009  *    http://www.apache.org/licenses/LICENSE-2.0
0010  *
0011  * Unless required by applicable law or agreed to in writing, software
0012  * distributed under the License is distributed on an "AS IS" BASIS,
0013  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
0014  * See the License for the specific language governing permissions and
0015  * limitations under the License.
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  * Server's response to client's challenge.
0030  *
0031  * Please see crypto/README.md for more details.
0032  */
0033 public class ServerResponse implements Encodable {
0034   /** Serialization tag used to catch incorrect payloads. */
0035   private static final byte TAG_BYTE = (byte) 0xFB;
0036 
0037   public final byte[] response;
0038   public final byte[] nonce;
0039   public final byte[] inputIv;
0040   public final byte[] outputIv;
0041 
0042   public ServerResponse(
0043       byte[] response,
0044       byte[] nonce,
0045       byte[] inputIv,
0046       byte[] outputIv) {
0047     this.response = response;
0048     this.nonce = nonce;
0049     this.inputIv = inputIv;
0050     this.outputIv = outputIv;
0051   }
0052 
0053   @Override
0054   public int encodedLength() {
0055     return 1 +
0056       Encoders.ByteArrays.encodedLength(response) +
0057       Encoders.ByteArrays.encodedLength(nonce) +
0058       Encoders.ByteArrays.encodedLength(inputIv) +
0059       Encoders.ByteArrays.encodedLength(outputIv);
0060   }
0061 
0062   @Override
0063   public void encode(ByteBuf buf) {
0064     buf.writeByte(TAG_BYTE);
0065     Encoders.ByteArrays.encode(buf, response);
0066     Encoders.ByteArrays.encode(buf, nonce);
0067     Encoders.ByteArrays.encode(buf, inputIv);
0068     Encoders.ByteArrays.encode(buf, outputIv);
0069   }
0070 
0071   public static ServerResponse decodeMessage(ByteBuffer buffer) {
0072     ByteBuf buf = Unpooled.wrappedBuffer(buffer);
0073 
0074     if (buf.readByte() != TAG_BYTE) {
0075       throw new IllegalArgumentException("Expected ServerResponse, received something else.");
0076     }
0077 
0078     return new ServerResponse(
0079       Encoders.ByteArrays.decode(buf),
0080       Encoders.ByteArrays.decode(buf),
0081       Encoders.ByteArrays.decode(buf),
0082       Encoders.ByteArrays.decode(buf));
0083   }
0084 
0085 }