View Javadoc

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.hbase.security.access;
20  
21  import java.io.ByteArrayInputStream;
22  import java.io.ByteArrayOutputStream;
23  import java.io.DataInput;
24  import java.io.DataOutput;
25  import java.io.IOException;
26  import java.io.InputStream;
27  import java.io.ObjectInputStream;
28  import java.io.ObjectOutputStream;
29  import java.io.Serializable;
30  import java.lang.reflect.Array;
31  import java.lang.reflect.InvocationTargetException;
32  import java.lang.reflect.Method;
33  import java.util.ArrayList;
34  import java.util.HashMap;
35  import java.util.List;
36  import java.util.Map;
37  import java.util.NavigableSet;
38  
39  import org.apache.commons.logging.Log;
40  import org.apache.commons.logging.LogFactory;
41  import org.apache.hadoop.hbase.classification.InterfaceAudience;
42  import org.apache.hadoop.conf.Configurable;
43  import org.apache.hadoop.conf.Configuration;
44  import org.apache.hadoop.conf.Configured;
45  import org.apache.hadoop.hbase.ClusterStatus;
46  import org.apache.hadoop.hbase.HColumnDescriptor;
47  import org.apache.hadoop.hbase.HConstants;
48  import org.apache.hadoop.hbase.HRegionInfo;
49  import org.apache.hadoop.hbase.HTableDescriptor;
50  import org.apache.hadoop.hbase.KeyValue;
51  import org.apache.hadoop.hbase.client.Action;
52  import org.apache.hadoop.hbase.client.Append;
53  import org.apache.hadoop.hbase.client.Delete;
54  import org.apache.hadoop.hbase.client.Get;
55  import org.apache.hadoop.hbase.client.Increment;
56  import org.apache.hadoop.hbase.client.MultiAction;
57  import org.apache.hadoop.hbase.client.MultiResponse;
58  import org.apache.hadoop.hbase.client.Put;
59  import org.apache.hadoop.hbase.client.Result;
60  import org.apache.hadoop.hbase.client.Row;
61  import org.apache.hadoop.hbase.client.RowMutations;
62  import org.apache.hadoop.hbase.client.Scan;
63  import org.apache.hadoop.hbase.filter.BinaryComparator;
64  import org.apache.hadoop.hbase.filter.BitComparator;
65  import org.apache.hadoop.hbase.filter.ByteArrayComparable;
66  import org.apache.hadoop.hbase.filter.ColumnCountGetFilter;
67  import org.apache.hadoop.hbase.filter.ColumnPrefixFilter;
68  import org.apache.hadoop.hbase.filter.ColumnRangeFilter;
69  import org.apache.hadoop.hbase.filter.CompareFilter;
70  import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
71  import org.apache.hadoop.hbase.filter.DependentColumnFilter;
72  import org.apache.hadoop.hbase.filter.FirstKeyOnlyFilter;
73  import org.apache.hadoop.hbase.filter.InclusiveStopFilter;
74  import org.apache.hadoop.hbase.filter.KeyOnlyFilter;
75  import org.apache.hadoop.hbase.filter.PageFilter;
76  import org.apache.hadoop.hbase.filter.PrefixFilter;
77  import org.apache.hadoop.hbase.filter.QualifierFilter;
78  import org.apache.hadoop.hbase.filter.RandomRowFilter;
79  import org.apache.hadoop.hbase.filter.RowFilter;
80  import org.apache.hadoop.hbase.filter.SingleColumnValueExcludeFilter;
81  import org.apache.hadoop.hbase.filter.SingleColumnValueFilter;
82  import org.apache.hadoop.hbase.filter.SkipFilter;
83  import org.apache.hadoop.hbase.filter.ValueFilter;
84  import org.apache.hadoop.hbase.filter.WhileMatchFilter;
85  import org.apache.hadoop.hbase.io.DataOutputOutputStream;
86  import org.apache.hadoop.hbase.io.WritableWithSize;
87  import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
88  import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
89  import org.apache.hadoop.hbase.regionserver.RegionOpeningState;
90  import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
91  import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
92  import org.apache.hadoop.hbase.wal.WAL.Entry;
93  import org.apache.hadoop.hbase.wal.WALKey;
94  import org.apache.hadoop.hbase.util.Bytes;
95  import org.apache.hadoop.hbase.util.ProtoUtil;
96  import org.apache.hadoop.io.MapWritable;
97  import org.apache.hadoop.io.Text;
98  import org.apache.hadoop.io.Writable;
99  import org.apache.hadoop.io.WritableFactories;
100 import org.apache.hadoop.io.WritableUtils;
101 
102 import com.google.protobuf.Message;
103 import com.google.protobuf.RpcController;
104 
105 /**
106  * <p>This is a customized version of the polymorphic hadoop
107  * {@link ObjectWritable}.  It removes UTF8 (HADOOP-414).
108  * Using {@link Text} intead of UTF-8 saves ~2% CPU between reading and writing
109  * objects running a short sequentialWrite Performance Evaluation test just in
110  * ObjectWritable alone; more when we're doing randomRead-ing.  Other
111  * optimizations include our passing codes for classes instead of the
112  * actual class names themselves.  This makes it so this class needs amendment
113  * if non-Writable classes are introduced -- if passed a Writable for which we
114  * have no code, we just do the old-school passing of the class name, etc. --
115  * but passing codes the  savings are large particularly when cell
116  * data is small (If < a couple of kilobytes, the encoding/decoding of class
117  * name and reflection to instantiate class was costing in excess of the cell
118  * handling).
119  * @deprecated This class is needed migrating TablePermissions written with
120  * Writables.  It is needed to read old permissions written pre-0.96.  This
121  * class is to be removed after HBase 0.96 ships since then all permissions
122  * will have been migrated and written with protobufs.
123  */
124 @Deprecated
125 @InterfaceAudience.Private
126 class HbaseObjectWritableFor96Migration implements Writable, WritableWithSize, Configurable {
127   protected final static Log LOG = LogFactory.getLog(HbaseObjectWritableFor96Migration.class);
128 
129   // Here we maintain two static maps of classes to code and vice versa.
130   // Add new classes+codes as wanted or figure way to auto-generate these
131   // maps.
132   static final Map<Integer, Class<?>> CODE_TO_CLASS =
133     new HashMap<Integer, Class<?>>();
134   static final Map<Class<?>, Integer> CLASS_TO_CODE =
135     new HashMap<Class<?>, Integer>();
136   // Special code that means 'not-encoded'; in this case we do old school
137   // sending of the class name using reflection, etc.
138   private static final byte NOT_ENCODED = 0;
139   //Generic array means that the array type is not one of the pre-defined arrays
140   //in the CLASS_TO_CODE map, but we have to still encode the array since it's
141   //elements are serializable by this class.
142   private static final int GENERIC_ARRAY_CODE;
143   private static final int NEXT_CLASS_CODE;
144   static {
145     ////////////////////////////////////////////////////////////////////////////
146     // WARNING: Please do not insert, remove or swap any line in this static  //
147     // block.  Doing so would change or shift all the codes used to serialize //
148     // objects, which makes backwards compatibility very hard for clients.    //
149     // New codes should always be added at the end. Code removal is           //
150     // discouraged because code is a short now.                               //
151     ////////////////////////////////////////////////////////////////////////////
152 
153     int code = NOT_ENCODED + 1;
154     // Primitive types.
155     addToMap(Boolean.TYPE, code++);
156     addToMap(Byte.TYPE, code++);
157     addToMap(Character.TYPE, code++);
158     addToMap(Short.TYPE, code++);
159     addToMap(Integer.TYPE, code++);
160     addToMap(Long.TYPE, code++);
161     addToMap(Float.TYPE, code++);
162     addToMap(Double.TYPE, code++);
163     addToMap(Void.TYPE, code++);
164 
165     // Other java types
166     addToMap(String.class, code++);
167     addToMap(byte [].class, code++);
168     addToMap(byte [][].class, code++);
169 
170     // Hadoop types
171     addToMap(Text.class, code++);
172     addToMap(Writable.class, code++);
173     addToMap(Writable [].class, code++);
174     code++; // Removed
175     addToMap(NullInstance.class, code++);
176 
177     // Hbase types
178     addToMap(HColumnDescriptor.class, code++);
179     addToMap(HConstants.Modify.class, code++);
180 
181     // We used to have a class named HMsg but its been removed.  Rather than
182     // just axe it, use following random Integer class -- we just chose any
183     // class from java.lang -- instead just so codes that follow stay
184     // in same relative place.
185     addToMap(Integer.class, code++);
186     addToMap(Integer[].class, code++);
187 
188     //HRegion shouldn't be pushed across the wire.
189     code++; //addToMap(HRegion.class, code++);
190     code++; //addToMap(HRegion[].class, code++);
191 
192     addToMap(HRegionInfo.class, code++);
193     addToMap(HRegionInfo[].class, code++);
194     code++; // Removed
195     code++; // Removed
196     addToMap(HTableDescriptor.class, code++);
197     addToMap(MapWritable.class, code++);
198 
199     //
200     // HBASE-880
201     //
202     addToMap(ClusterStatus.class, code++);
203     addToMap(Delete.class, code++);
204     addToMap(Get.class, code++);
205     addToMap(KeyValue.class, code++);
206     addToMap(KeyValue[].class, code++);
207     addToMap(Put.class, code++);
208     addToMap(Put[].class, code++);
209     addToMap(Result.class, code++);
210     addToMap(Result[].class, code++);
211     addToMap(Scan.class, code++);
212 
213     addToMap(WhileMatchFilter.class, code++);
214     addToMap(PrefixFilter.class, code++);
215     addToMap(PageFilter.class, code++);
216     addToMap(InclusiveStopFilter.class, code++);
217     addToMap(ColumnCountGetFilter.class, code++);
218     addToMap(SingleColumnValueFilter.class, code++);
219     addToMap(SingleColumnValueExcludeFilter.class, code++);
220     addToMap(BinaryComparator.class, code++);
221     addToMap(BitComparator.class, code++);
222     addToMap(CompareFilter.class, code++);
223     addToMap(RowFilter.class, code++);
224     addToMap(ValueFilter.class, code++);
225     addToMap(QualifierFilter.class, code++);
226     addToMap(SkipFilter.class, code++);
227     addToMap(ByteArrayComparable.class, code++);
228     addToMap(FirstKeyOnlyFilter.class, code++);
229     addToMap(DependentColumnFilter.class, code++);
230 
231     addToMap(Delete [].class, code++);
232 
233     addToMap(Entry.class, code++);
234     addToMap(Entry[].class, code++);
235     addToMap(HLogKey.class, code++);
236 
237     addToMap(List.class, code++);
238 
239     addToMap(NavigableSet.class, code++);
240     addToMap(ColumnPrefixFilter.class, code++);
241 
242     // Multi
243     addToMap(Row.class, code++);
244     addToMap(Action.class, code++);
245     addToMap(MultiAction.class, code++);
246     addToMap(MultiResponse.class, code++);
247 
248     // coprocessor execution
249     // Exec no longer exists --> addToMap(Exec.class, code++);
250     code++;
251     addToMap(Increment.class, code++);
252 
253     addToMap(KeyOnlyFilter.class, code++);
254 
255     // serializable
256     addToMap(Serializable.class, code++);
257 
258     addToMap(RandomRowFilter.class, code++);
259 
260     addToMap(CompareOp.class, code++);
261 
262     addToMap(ColumnRangeFilter.class, code++);
263 
264     // HServerLoad no longer exists; increase code so other classes stay the same.
265     code++;
266     //addToMap(HServerLoad.class, code++);
267 
268     addToMap(RegionOpeningState.class, code++);
269 
270     addToMap(HTableDescriptor[].class, code++);
271 
272     addToMap(Append.class, code++);
273 
274     addToMap(RowMutations.class, code++);
275 
276     addToMap(Message.class, code++);
277 
278     //java.lang.reflect.Array is a placeholder for arrays not defined above
279     GENERIC_ARRAY_CODE = code++;
280     addToMap(Array.class, GENERIC_ARRAY_CODE);
281 
282     addToMap(RpcController.class, code++);
283 
284     // make sure that this is the last statement in this static block
285     NEXT_CLASS_CODE = code;
286   }
287 
288   private Class<?> declaredClass;
289   private Object instance;
290   private Configuration conf;
291 
292   /** default constructor for writable */
293   HbaseObjectWritableFor96Migration() {
294     super();
295   }
296 
297   /**
298    * @param instance
299    */
300   HbaseObjectWritableFor96Migration(Object instance) {
301     set(instance);
302   }
303 
304   /**
305    * @param declaredClass
306    * @param instance
307    */
308   HbaseObjectWritableFor96Migration(Class<?> declaredClass, Object instance) {
309     this.declaredClass = declaredClass;
310     this.instance = instance;
311   }
312 
313   /** @return the instance, or null if none. */
314   Object get() { return instance; }
315 
316   /** @return the class this is meant to be. */
317   Class<?> getDeclaredClass() { return declaredClass; }
318 
319   /**
320    * Reset the instance.
321    * @param instance
322    */
323   void set(Object instance) {
324     this.declaredClass = instance.getClass();
325     this.instance = instance;
326   }
327 
328   /**
329    * @see java.lang.Object#toString()
330    */
331   @Override
332   public String toString() {
333     return "OW[class=" + declaredClass + ",value=" + instance + "]";
334   }
335 
336 
337   public void readFields(DataInput in) throws IOException {
338     readObject(in, this, this.conf);
339   }
340 
341   public void write(DataOutput out) throws IOException {
342     writeObject(out, instance, declaredClass, conf);
343   }
344 
345   public long getWritableSize() {
346     return getWritableSize(instance, declaredClass, conf);
347   }
348 
349   private static class NullInstance extends Configured implements Writable {
350     Class<?> declaredClass;
351     /** default constructor for writable */
352     @SuppressWarnings("unused")
353     public NullInstance() { super(null); }
354 
355     /**
356      * @param declaredClass
357      * @param conf
358      */
359     public NullInstance(Class<?> declaredClass, Configuration conf) {
360       super(conf);
361       this.declaredClass = declaredClass;
362     }
363 
364     public void readFields(DataInput in) throws IOException {
365       this.declaredClass = CODE_TO_CLASS.get(WritableUtils.readVInt(in));
366     }
367 
368     public void write(DataOutput out) throws IOException {
369       writeClassCode(out, this.declaredClass);
370     }
371   }
372 
373   static Integer getClassCode(final Class<?> c)
374   throws IOException {
375     Integer code = CLASS_TO_CODE.get(c);
376     if (code == null ) {
377       if (List.class.isAssignableFrom(c)) {
378         code = CLASS_TO_CODE.get(List.class);
379       } else if (Writable.class.isAssignableFrom(c)) {
380         code = CLASS_TO_CODE.get(Writable.class);
381       } else if (c.isArray()) {
382         code = CLASS_TO_CODE.get(Array.class);
383       } else if (Message.class.isAssignableFrom(c)) {
384         code = CLASS_TO_CODE.get(Message.class);
385       } else if (Serializable.class.isAssignableFrom(c)){
386         code = CLASS_TO_CODE.get(Serializable.class);
387       } else if (Scan.class.isAssignableFrom(c)) {
388         code = CLASS_TO_CODE.get(Scan.class);
389       }
390     }
391     return code;
392   }
393 
394   /**
395    * @return the next object code in the list.  Used in testing to verify that additional fields are not added 
396    */
397   static int getNextClassCode(){
398     return NEXT_CLASS_CODE;
399   }
400 
401   /**
402    * Write out the code for passed Class.
403    * @param out
404    * @param c
405    * @throws IOException
406    */
407   static void writeClassCode(final DataOutput out, final Class<?> c)
408       throws IOException {
409     Integer code = getClassCode(c);
410 
411     if (code == null) {
412       LOG.error("Unsupported type " + c);
413       StackTraceElement[] els = new Exception().getStackTrace();
414       for(StackTraceElement elem : els) {
415         LOG.error(elem.getMethodName());
416       }
417       throw new UnsupportedOperationException("No code for unexpected " + c);
418     }
419     WritableUtils.writeVInt(out, code);
420   }
421 
422   static long getWritableSize(Object instance, Class declaredClass,
423                                      Configuration conf) {
424     return 0L; // no hint is the default.
425   }
426   /**
427    * Write a {@link Writable}, {@link String}, primitive type, or an array of
428    * the preceding.
429    * @param out
430    * @param instance
431    * @param declaredClass
432    * @param conf
433    * @throws IOException
434    */
435   @SuppressWarnings("unchecked")
436   static void writeObject(DataOutput out, Object instance,
437                                  Class declaredClass,
438                                  Configuration conf)
439   throws IOException {
440 
441     Object instanceObj = instance;
442     Class declClass = declaredClass;
443 
444     if (instanceObj == null) {                       // null
445       instanceObj = new NullInstance(declClass, conf);
446       declClass = Writable.class;
447     }
448     writeClassCode(out, declClass);
449     if (declClass.isArray()) {                // array
450       // If bytearray, just dump it out -- avoid the recursion and
451       // byte-at-a-time we were previously doing.
452       if (declClass.equals(byte [].class)) {
453         Bytes.writeByteArray(out, (byte [])instanceObj);
454       } else {
455         //if it is a Generic array, write the element's type
456         if (getClassCode(declaredClass) == GENERIC_ARRAY_CODE) {
457           Class<?> componentType = declaredClass.getComponentType();
458           writeClass(out, componentType);
459         }
460 
461         int length = Array.getLength(instanceObj);
462         out.writeInt(length);
463         for (int i = 0; i < length; i++) {
464           Object item = Array.get(instanceObj, i);
465           writeObject(out, item,
466                     item.getClass(), conf);
467         }
468       }
469     } else if (List.class.isAssignableFrom(declClass)) {
470       List list = (List)instanceObj;
471       int length = list.size();
472       out.writeInt(length);
473       for (int i = 0; i < length; i++) {
474         Object elem = list.get(i);
475         writeObject(out, elem,
476                   elem == null ? Writable.class : elem.getClass(), conf);
477       }
478     } else if (declClass == String.class) {   // String
479       Text.writeString(out, (String)instanceObj);
480     } else if (declClass.isPrimitive()) {     // primitive type
481       if (declClass == Boolean.TYPE) {        // boolean
482         out.writeBoolean(((Boolean)instanceObj).booleanValue());
483       } else if (declClass == Character.TYPE) { // char
484         out.writeChar(((Character)instanceObj).charValue());
485       } else if (declClass == Byte.TYPE) {    // byte
486         out.writeByte(((Byte)instanceObj).byteValue());
487       } else if (declClass == Short.TYPE) {   // short
488         out.writeShort(((Short)instanceObj).shortValue());
489       } else if (declClass == Integer.TYPE) { // int
490         out.writeInt(((Integer)instanceObj).intValue());
491       } else if (declClass == Long.TYPE) {    // long
492         out.writeLong(((Long)instanceObj).longValue());
493       } else if (declClass == Float.TYPE) {   // float
494         out.writeFloat(((Float)instanceObj).floatValue());
495       } else if (declClass == Double.TYPE) {  // double
496         out.writeDouble(((Double)instanceObj).doubleValue());
497       } else if (declClass == Void.TYPE) {    // void
498       } else {
499         throw new IllegalArgumentException("Not a primitive: "+declClass);
500       }
501     } else if (declClass.isEnum()) {         // enum
502       Text.writeString(out, ((Enum)instanceObj).name());
503     } else if (Message.class.isAssignableFrom(declaredClass)) {
504       Text.writeString(out, instanceObj.getClass().getName());
505       ((Message)instance).writeDelimitedTo(
506           DataOutputOutputStream.constructOutputStream(out));
507     } else if (Writable.class.isAssignableFrom(declClass)) { // Writable
508       Class <?> c = instanceObj.getClass();
509       Integer code = CLASS_TO_CODE.get(c);
510       if (code == null) {
511         out.writeByte(NOT_ENCODED);
512         Text.writeString(out, c.getName());
513       } else {
514         writeClassCode(out, c);
515       }
516       ((Writable)instanceObj).write(out);
517     } else if (Serializable.class.isAssignableFrom(declClass)) {
518       Class <?> c = instanceObj.getClass();
519       Integer code = CLASS_TO_CODE.get(c);
520       if (code == null) {
521         out.writeByte(NOT_ENCODED);
522         Text.writeString(out, c.getName());
523       } else {
524         writeClassCode(out, c);
525       }
526       ByteArrayOutputStream bos = null;
527       ObjectOutputStream oos = null;
528       try{
529         bos = new ByteArrayOutputStream();
530         oos = new ObjectOutputStream(bos);
531         oos.writeObject(instanceObj);
532         byte[] value = bos.toByteArray();
533         out.writeInt(value.length);
534         out.write(value);
535       } finally {
536         if(bos!=null) bos.close();
537         if(oos!=null) oos.close();
538       }
539     } else if (Scan.class.isAssignableFrom(declClass)) {
540       Scan scan = (Scan)instanceObj;
541       byte [] scanBytes = ProtobufUtil.toScan(scan).toByteArray();
542       out.writeInt(scanBytes.length);
543       out.write(scanBytes);
544     } else if (Entry.class.isAssignableFrom(declClass)) {
545       // Entry is no longer Writable, maintain compatible serialization.
546       // Writables write their exact runtime class
547       Class <?> c = instanceObj.getClass();
548       Integer code = CLASS_TO_CODE.get(c);
549       if (code == null) {
550         out.writeByte(NOT_ENCODED);
551         Text.writeString(out, c.getName());
552       } else {
553         writeClassCode(out, c);
554       }
555       final Entry entry = (Entry)instanceObj;
556       // We only support legacy HLogKey
557       WALKey key = entry.getKey();
558       if (!(key instanceof HLogKey)) {
559         throw new IOException("Can't write Entry '" + instanceObj + "' due to key class '" +
560             key.getClass() + "'");
561       }
562       ((HLogKey)key).write(out);
563       entry.getEdit().write(out);
564     } else {
565       throw new IOException("Can't write: "+instanceObj+" as "+declClass);
566     }
567   }
568 
569   /** Writes the encoded class code as defined in CLASS_TO_CODE, or
570    * the whole class name if not defined in the mapping.
571    */
572   static void writeClass(DataOutput out, Class<?> c) throws IOException {
573     Integer code = CLASS_TO_CODE.get(c);
574     if (code == null) {
575       WritableUtils.writeVInt(out, NOT_ENCODED);
576       Text.writeString(out, c.getName());
577     } else {
578       WritableUtils.writeVInt(out, code);
579     }
580   }
581 
582   /** Reads and returns the class as written by {@link #writeClass(DataOutput, Class)} */
583   static Class<?> readClass(Configuration conf, DataInput in) throws IOException {
584     Class<?> instanceClass = null;
585     int b = (byte)WritableUtils.readVInt(in);
586     if (b == NOT_ENCODED) {
587       String className = Text.readString(in);
588       try {
589         instanceClass = getClassByName(conf, className);
590       } catch (ClassNotFoundException e) {
591         LOG.error("Can't find class " + className, e);
592         throw new IOException("Can't find class " + className, e);
593       }
594     } else {
595       instanceClass = CODE_TO_CLASS.get(b);
596     }
597     return instanceClass;
598   }
599 
600   /**
601    * Read a {@link Writable}, {@link String}, primitive type, or an array of
602    * the preceding.
603    * @param in
604    * @param conf
605    * @return the object
606    * @throws IOException
607    */
608   static Object readObject(DataInput in, Configuration conf)
609     throws IOException {
610     return readObject(in, null, conf);
611   }
612 
613   /**
614    * Read a {@link Writable}, {@link String}, primitive type, or an array of
615    * the preceding.
616    * @param in
617    * @param objectWritable
618    * @param conf
619    * @return the object
620    * @throws IOException
621    */
622   @SuppressWarnings("unchecked")
623   static Object readObject(DataInput in,
624       HbaseObjectWritableFor96Migration objectWritable, Configuration conf)
625   throws IOException {
626     Class<?> declaredClass = CODE_TO_CLASS.get(WritableUtils.readVInt(in));
627     Object instance;
628     if (declaredClass.isPrimitive()) {            // primitive types
629       if (declaredClass == Boolean.TYPE) {             // boolean
630         instance = Boolean.valueOf(in.readBoolean());
631       } else if (declaredClass == Character.TYPE) {    // char
632         instance = Character.valueOf(in.readChar());
633       } else if (declaredClass == Byte.TYPE) {         // byte
634         instance = Byte.valueOf(in.readByte());
635       } else if (declaredClass == Short.TYPE) {        // short
636         instance = Short.valueOf(in.readShort());
637       } else if (declaredClass == Integer.TYPE) {      // int
638         instance = Integer.valueOf(in.readInt());
639       } else if (declaredClass == Long.TYPE) {         // long
640         instance = Long.valueOf(in.readLong());
641       } else if (declaredClass == Float.TYPE) {        // float
642         instance = Float.valueOf(in.readFloat());
643       } else if (declaredClass == Double.TYPE) {       // double
644         instance = Double.valueOf(in.readDouble());
645       } else if (declaredClass == Void.TYPE) {         // void
646         instance = null;
647       } else {
648         throw new IllegalArgumentException("Not a primitive: "+declaredClass);
649       }
650     } else if (declaredClass.isArray()) {              // array
651       if (declaredClass.equals(byte [].class)) {
652         instance = Bytes.readByteArray(in);
653       } else {
654         int length = in.readInt();
655         instance = Array.newInstance(declaredClass.getComponentType(), length);
656         for (int i = 0; i < length; i++) {
657           Array.set(instance, i, readObject(in, conf));
658         }
659       }
660     } else if (declaredClass.equals(Array.class)) { //an array not declared in CLASS_TO_CODE
661       Class<?> componentType = readClass(conf, in);
662       int length = in.readInt();
663       instance = Array.newInstance(componentType, length);
664       for (int i = 0; i < length; i++) {
665         Array.set(instance, i, readObject(in, conf));
666       }
667     } else if (List.class.isAssignableFrom(declaredClass)) {            // List
668       int length = in.readInt();
669       instance = new ArrayList(length);
670       for (int i = 0; i < length; i++) {
671         ((ArrayList)instance).add(readObject(in, conf));
672       }
673     } else if (declaredClass == String.class) {        // String
674       instance = Text.readString(in);
675     } else if (declaredClass.isEnum()) {         // enum
676       instance = Enum.valueOf((Class<? extends Enum>) declaredClass,
677         Text.readString(in));
678     } else if (declaredClass == Message.class) {
679       String className = Text.readString(in);
680       try {
681         declaredClass = getClassByName(conf, className);
682         instance = tryInstantiateProtobuf(declaredClass, in);
683       } catch (ClassNotFoundException e) {
684         LOG.error("Can't find class " + className, e);
685         throw new IOException("Can't find class " + className, e);
686       }
687     } else if (Scan.class.isAssignableFrom(declaredClass)) {
688       int length = in.readInt();
689       byte [] scanBytes = new byte[length];
690       in.readFully(scanBytes);
691       ClientProtos.Scan.Builder scanProto = ClientProtos.Scan.newBuilder();
692       ProtobufUtil.mergeFrom(scanProto, scanBytes);
693       instance = ProtobufUtil.toScan(scanProto.build());
694     } else {                                      // Writable or Serializable
695       Class instanceClass = null;
696       int b = (byte)WritableUtils.readVInt(in);
697       if (b == NOT_ENCODED) {
698         String className = Text.readString(in);
699         if ("org.apache.hadoop.hbase.regionserver.wal.HLog$Entry".equals(className)) {
700           className = Entry.class.getName();
701         }
702         try {
703           instanceClass = getClassByName(conf, className);
704         } catch (ClassNotFoundException e) {
705           LOG.error("Can't find class " + className, e);
706           throw new IOException("Can't find class " + className, e);
707         }
708       } else {
709         instanceClass = CODE_TO_CLASS.get(b);
710       }
711       if(Writable.class.isAssignableFrom(instanceClass)){
712         Writable writable = WritableFactories.newInstance(instanceClass, conf);
713         try {
714           writable.readFields(in);
715         } catch (Exception e) {
716           LOG.error("Error in readFields", e);
717           throw new IOException("Error in readFields" , e);
718         }
719         instance = writable;
720         if (instanceClass == NullInstance.class) {  // null
721           declaredClass = ((NullInstance)instance).declaredClass;
722           instance = null;
723         }
724       } else if (Entry.class.isAssignableFrom(instanceClass)) {
725         // Entry stopped being Writable; maintain serialization support.
726         final HLogKey key = new HLogKey();
727         final WALEdit edit = new WALEdit();
728         key.readFields(in);
729         edit.readFields(in);
730         instance = new Entry(key, edit);
731       } else {
732         int length = in.readInt();
733         byte[] objectBytes = new byte[length];
734         in.readFully(objectBytes);
735         ByteArrayInputStream bis = null;
736         ObjectInputStream ois = null;
737         try {
738           bis = new ByteArrayInputStream(objectBytes);
739           ois = new ObjectInputStream(bis);
740           instance = ois.readObject();
741         } catch (ClassNotFoundException e) {
742           LOG.error("Class not found when attempting to deserialize object", e);
743           throw new IOException("Class not found when attempting to " +
744               "deserialize object", e);
745         } finally {
746           if(bis!=null) bis.close();
747           if(ois!=null) ois.close();
748         }
749       }
750     }
751     if (objectWritable != null) {                 // store values
752       objectWritable.declaredClass = declaredClass;
753       objectWritable.instance = instance;
754     }
755     return instance;
756   }
757 
758   /**
759    * Try to instantiate a protocol buffer of the given message class
760    * from the given input stream.
761    *
762    * @param protoClass the class of the generated protocol buffer
763    * @param dataIn the input stream to read from
764    * @return the instantiated Message instance
765    * @throws IOException if an IO problem occurs
766    */
767   static Message tryInstantiateProtobuf(
768       Class<?> protoClass,
769       DataInput dataIn) throws IOException {
770 
771     try {
772       if (dataIn instanceof InputStream) {
773         // We can use the built-in parseDelimitedFrom and not have to re-copy
774         // the data
775         Method parseMethod = getStaticProtobufMethod(protoClass,
776             "parseDelimitedFrom", InputStream.class);
777         return (Message)parseMethod.invoke(null, (InputStream)dataIn);
778       } else {
779         // Have to read it into a buffer first, since protobuf doesn't deal
780         // with the DataInput interface directly.
781 
782         // Read the size delimiter that writeDelimitedTo writes
783         int size = ProtoUtil.readRawVarint32(dataIn);
784         if (size < 0) {
785           throw new IOException("Invalid size: " + size);
786         }
787 
788         byte[] data = new byte[size];
789         dataIn.readFully(data);
790         Method parseMethod = getStaticProtobufMethod(protoClass,
791             "parseFrom", byte[].class);
792         return (Message)parseMethod.invoke(null, data);
793       }
794     } catch (InvocationTargetException e) {
795 
796       if (e.getCause() instanceof IOException) {
797         throw (IOException)e.getCause();
798       } else {
799         throw new IOException(e.getCause());
800       }
801     } catch (IllegalAccessException iae) {
802       throw new AssertionError("Could not access parse method in " +
803           protoClass);
804     }
805   }
806 
807   static Method getStaticProtobufMethod(Class<?> declaredClass, String method,
808       Class<?> ... args) {
809 
810     try {
811       return declaredClass.getMethod(method, args);
812     } catch (Exception e) {
813       // This is a bug in Hadoop - protobufs should all have this static method
814       throw new AssertionError("Protocol buffer class " + declaredClass +
815           " does not have an accessible parseFrom(InputStream) method!");
816     }
817   }
818 
819   @SuppressWarnings("unchecked")
820   private static Class getClassByName(Configuration conf, String className)
821   throws ClassNotFoundException {
822     if(conf != null) {
823       return conf.getClassByName(className);
824     }
825     ClassLoader cl = Thread.currentThread().getContextClassLoader();
826     if(cl == null) {
827       cl = HbaseObjectWritableFor96Migration.class.getClassLoader();
828     }
829     return Class.forName(className, true, cl);
830   }
831 
832   private static void addToMap(final Class<?> clazz, final int code) {
833     CLASS_TO_CODE.put(clazz, code);
834     CODE_TO_CLASS.put(code, clazz);
835   }
836 
837   public void setConf(Configuration conf) {
838     this.conf = conf;
839   }
840 
841   public Configuration getConf() {
842     return this.conf;
843   }
844 }