1 /**
2  * Licensed to the Apache Software Foundation (ASF) under one
3  * or more contributor license agreements.  See the NOTICE file
4  * distributed with this work for additional information
5  * regarding copyright ownership.  The ASF licenses this file
6  * to you under the Apache License, Version 2.0 (the
7  * "License"); you may not use this file except in compliance
8  * with the License.  You may obtain a copy of the License at
9  *
10  *     http://www.apache.org/licenses/LICENSE-2.0
11  *
12  * Unless required by applicable law or agreed to in writing, software
13  * distributed under the License is distributed on an "AS IS" BASIS,
14  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15  * See the License for the specific language governing permissions and
16  * limitations under the License.
17  */
18 
19 package org.apache.hadoop.ipc;
20 
21 import java.io.DataInput;
22 import java.io.DataOutput;
23 import java.io.IOException;
24 import java.lang.reflect.Method;
25 import java.util.Arrays;
26 import java.util.HashMap;
27 
28 import org.apache.hadoop.io.Writable;
29 import org.apache.hadoop.io.WritableFactories;
30 import org.apache.hadoop.io.WritableFactory;
31 
32 import com.google.common.annotations.VisibleForTesting;
33 
34 public class ProtocolSignature implements Writable {
35   static {               // register a ctor
WritableFactories.setFactory(ProtocolSignature.class, new WritableFactory() { @Override public Writable newInstance() { return new ProtocolSignature(); } })36     WritableFactories.setFactory
37       (ProtocolSignature.class,
38        new WritableFactory() {
39          @Override
40         public Writable newInstance() { return new ProtocolSignature(); }
41        });
42   }
43 
44   private long version;
45   private int[] methods = null; // an array of method hash codes
46 
47   /**
48    * default constructor
49    */
ProtocolSignature()50   public ProtocolSignature() {
51   }
52 
53   /**
54    * Constructor
55    *
56    * @param version server version
57    * @param methodHashcodes hash codes of the methods supported by server
58    */
ProtocolSignature(long version, int[] methodHashcodes)59   public ProtocolSignature(long version, int[] methodHashcodes) {
60     this.version = version;
61     this.methods = methodHashcodes;
62   }
63 
getVersion()64   public long getVersion() {
65     return version;
66   }
67 
getMethods()68   public int[] getMethods() {
69     return methods;
70   }
71 
72   @Override
readFields(DataInput in)73   public void readFields(DataInput in) throws IOException {
74     version = in.readLong();
75     boolean hasMethods = in.readBoolean();
76     if (hasMethods) {
77       int numMethods = in.readInt();
78       methods = new int[numMethods];
79       for (int i=0; i<numMethods; i++) {
80         methods[i] = in.readInt();
81       }
82     }
83   }
84 
85   @Override
write(DataOutput out)86   public void write(DataOutput out) throws IOException {
87     out.writeLong(version);
88     if (methods == null) {
89       out.writeBoolean(false);
90     } else {
91       out.writeBoolean(true);
92       out.writeInt(methods.length);
93       for (int method : methods) {
94         out.writeInt(method);
95       }
96     }
97   }
98 
99   /**
100    * Calculate a method's hash code considering its method
101    * name, returning type, and its parameter types
102    *
103    * @param method a method
104    * @return its hash code
105    */
getFingerprint(Method method)106   static int getFingerprint(Method method) {
107     int hashcode = method.getName().hashCode();
108     hashcode =  hashcode + 31*method.getReturnType().getName().hashCode();
109     for (Class<?> type : method.getParameterTypes()) {
110       hashcode = 31*hashcode ^ type.getName().hashCode();
111     }
112     return hashcode;
113   }
114 
115   /**
116    * Convert an array of Method into an array of hash codes
117    *
118    * @param methods
119    * @return array of hash codes
120    */
getFingerprints(Method[] methods)121   private static int[] getFingerprints(Method[] methods) {
122     if (methods == null) {
123       return null;
124     }
125     int[] hashCodes = new int[methods.length];
126     for (int i = 0; i<methods.length; i++) {
127       hashCodes[i] = getFingerprint(methods[i]);
128     }
129     return hashCodes;
130   }
131 
132   /**
133    * Get the hash code of an array of methods
134    * Methods are sorted before hashcode is calculated.
135    * So the returned value is irrelevant of the method order in the array.
136    *
137    * @param methods an array of methods
138    * @return the hash code
139    */
getFingerprint(Method[] methods)140   static int getFingerprint(Method[] methods) {
141     return getFingerprint(getFingerprints(methods));
142   }
143 
144   /**
145    * Get the hash code of an array of hashcodes
146    * Hashcodes are sorted before hashcode is calculated.
147    * So the returned value is irrelevant of the hashcode order in the array.
148    *
149    * @param methods an array of methods
150    * @return the hash code
151    */
getFingerprint(int[] hashcodes)152   static int getFingerprint(int[] hashcodes) {
153     Arrays.sort(hashcodes);
154     return Arrays.hashCode(hashcodes);
155 
156   }
157   private static class ProtocolSigFingerprint {
158     private ProtocolSignature signature;
159     private int fingerprint;
160 
ProtocolSigFingerprint(ProtocolSignature sig, int fingerprint)161     ProtocolSigFingerprint(ProtocolSignature sig, int fingerprint) {
162       this.signature = sig;
163       this.fingerprint = fingerprint;
164     }
165   }
166 
167   /**
168    * A cache that maps a protocol's name to its signature & finger print
169    */
170   private final static HashMap<String, ProtocolSigFingerprint>
171      PROTOCOL_FINGERPRINT_CACHE =
172        new HashMap<String, ProtocolSigFingerprint>();
173 
174   @VisibleForTesting
resetCache()175   public static void resetCache() {
176     PROTOCOL_FINGERPRINT_CACHE.clear();
177   }
178 
179   /**
180    * Return a protocol's signature and finger print from cache
181    *
182    * @param protocol a protocol class
183    * @param serverVersion protocol version
184    * @return its signature and finger print
185    */
getSigFingerprint( Class <?> protocol, long serverVersion)186   private static ProtocolSigFingerprint getSigFingerprint(
187       Class <?> protocol, long serverVersion) {
188     String protocolName = RPC.getProtocolName(protocol);
189     synchronized (PROTOCOL_FINGERPRINT_CACHE) {
190       ProtocolSigFingerprint sig = PROTOCOL_FINGERPRINT_CACHE.get(protocolName);
191       if (sig == null) {
192         int[] serverMethodHashcodes = getFingerprints(protocol.getMethods());
193         sig = new ProtocolSigFingerprint(
194             new ProtocolSignature(serverVersion, serverMethodHashcodes),
195             getFingerprint(serverMethodHashcodes));
196         PROTOCOL_FINGERPRINT_CACHE.put(protocolName, sig);
197       }
198       return sig;
199     }
200   }
201 
202   /**
203    * Get a server protocol's signature
204    *
205    * @param clientMethodsHashCode client protocol methods hashcode
206    * @param serverVersion server protocol version
207    * @param protocol protocol
208    * @return the server's protocol signature
209    */
getProtocolSignature( int clientMethodsHashCode, long serverVersion, Class<? extends VersionedProtocol> protocol)210   public static ProtocolSignature getProtocolSignature(
211       int clientMethodsHashCode,
212       long serverVersion,
213       Class<? extends VersionedProtocol> protocol) {
214     // try to get the finger print & signature from the cache
215     ProtocolSigFingerprint sig = getSigFingerprint(protocol, serverVersion);
216 
217     // check if the client side protocol matches the one on the server side
218     if (clientMethodsHashCode == sig.fingerprint) {
219       return new ProtocolSignature(serverVersion, null);  // null indicates a match
220     }
221 
222     return sig.signature;
223   }
224 
getProtocolSignature(String protocolName, long version)225   public static ProtocolSignature getProtocolSignature(String protocolName,
226       long version) throws ClassNotFoundException {
227     Class<?> protocol = Class.forName(protocolName);
228     return getSigFingerprint(protocol, version).signature;
229   }
230 
231   /**
232    * Get a server protocol's signature
233    *
234    * @param server server implementation
235    * @param protocol server protocol
236    * @param clientVersion client's version
237    * @param clientMethodsHash client's protocol's hash code
238    * @return the server protocol's signature
239    * @throws IOException if any error occurs
240    */
241   @SuppressWarnings("unchecked")
getProtocolSignature(VersionedProtocol server, String protocol, long clientVersion, int clientMethodsHash)242   public static ProtocolSignature getProtocolSignature(VersionedProtocol server,
243       String protocol,
244       long clientVersion, int clientMethodsHash) throws IOException {
245     Class<? extends VersionedProtocol> inter;
246     try {
247       inter = (Class<? extends VersionedProtocol>)Class.forName(protocol);
248     } catch (Exception e) {
249       throw new IOException(e);
250     }
251     long serverVersion = server.getProtocolVersion(protocol, clientVersion);
252     return ProtocolSignature.getProtocolSignature(
253         clientMethodsHash, serverVersion, inter);
254   }
255 }
256