View Javadoc

1   /**
2    *
3    * Licensed to the Apache Software Foundation (ASF) under one
4    * or more contributor license agreements.  See the NOTICE file
5    * distributed with this work for additional information
6    * regarding copyright ownership.  The ASF licenses this file
7    * to you under the Apache License, Version 2.0 (the
8    * "License"); you may not use this file except in compliance
9    * with the License.  You may obtain a copy of the License at
10   *
11   *     http://www.apache.org/licenses/LICENSE-2.0
12   *
13   * Unless required by applicable law or agreed to in writing, software
14   * distributed under the License is distributed on an "AS IS" BASIS,
15   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
16   * See the License for the specific language governing permissions and
17   * limitations under the License.
18   */
19  package org.apache.hadoop.hbase;
20  
21  import java.io.DataInput;
22  import java.io.DataOutput;
23  import java.io.IOException;
24  import java.util.Collections;
25  import java.util.HashMap;
26  import java.util.HashSet;
27  import java.util.Map;
28  import java.util.Set;
29  
30  import org.apache.hadoop.hbase.classification.InterfaceAudience;
31  import org.apache.hadoop.hbase.classification.InterfaceStability;
32  import org.apache.hadoop.hbase.exceptions.DeserializationException;
33  import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
34  import org.apache.hadoop.hbase.io.compress.Compression;
35  import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
36  import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
37  import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPair;
38  import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ColumnFamilySchema;
39  import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameStringPair;
40  import org.apache.hadoop.hbase.regionserver.BloomType;
41  import org.apache.hadoop.hbase.util.Bytes;
42  import org.apache.hadoop.hbase.util.PrettyPrinter;
43  import org.apache.hadoop.hbase.util.PrettyPrinter.Unit;
44  import org.apache.hadoop.io.Text;
45  import org.apache.hadoop.io.WritableComparable;
46  
47  import com.google.common.base.Preconditions;
48  import org.apache.hadoop.hbase.util.ByteStringer;
49  
50  /**
51   * An HColumnDescriptor contains information about a column family such as the
52   * number of versions, compression settings, etc.
53   *
54   * It is used as input when creating a table or adding a column.
55   */
56  @InterfaceAudience.Public
57  @InterfaceStability.Evolving
58  public class HColumnDescriptor implements WritableComparable<HColumnDescriptor> {
59    // For future backward compatibility
60  
61    // Version  3 was when column names become byte arrays and when we picked up
62    // Time-to-live feature.  Version 4 was when we moved to byte arrays, HBASE-82.
63    // Version  5 was when bloom filter descriptors were removed.
64    // Version  6 adds metadata as a map where keys and values are byte[].
65    // Version  7 -- add new compression and hfile blocksize to HColumnDescriptor (HBASE-1217)
66    // Version  8 -- reintroduction of bloom filters, changed from boolean to enum
67    // Version  9 -- add data block encoding
68    // Version 10 -- change metadata to standard type.
69    // Version 11 -- add column family level configuration.
70    private static final byte COLUMN_DESCRIPTOR_VERSION = (byte) 11;
71  
72    // These constants are used as FileInfo keys
73    public static final String COMPRESSION = "COMPRESSION";
74    public static final String COMPRESSION_COMPACT = "COMPRESSION_COMPACT";
75    public static final String ENCODE_ON_DISK = // To be removed, it is not used anymore
76        "ENCODE_ON_DISK";
77    public static final String DATA_BLOCK_ENCODING =
78        "DATA_BLOCK_ENCODING";
79    /**
80     * Key for the BLOCKCACHE attribute.
81     * A more exact name would be CACHE_DATA_ON_READ because this flag sets whether or not we
82     * cache DATA blocks.  We always cache INDEX and BLOOM blocks; caching these blocks cannot be
83     * disabled.
84     */
85    public static final String BLOCKCACHE = "BLOCKCACHE";
86    public static final String CACHE_DATA_ON_WRITE = "CACHE_DATA_ON_WRITE";
87    public static final String CACHE_INDEX_ON_WRITE = "CACHE_INDEX_ON_WRITE";
88    public static final String CACHE_BLOOMS_ON_WRITE = "CACHE_BLOOMS_ON_WRITE";
89    public static final String EVICT_BLOCKS_ON_CLOSE = "EVICT_BLOCKS_ON_CLOSE";
90    /**
91     * Key for cache data into L1 if cache is set up with more than one tier.
92     * To set in the shell, do something like this:
93     * <code>hbase(main):003:0> create 't',
94     *    {NAME => 't', CONFIGURATION => {CACHE_DATA_IN_L1 => 'true'}}</code>
95     */
96    public static final String CACHE_DATA_IN_L1 = "CACHE_DATA_IN_L1";
97  
98    /**
99     * Key for the PREFETCH_BLOCKS_ON_OPEN attribute.
100    * If set, all INDEX, BLOOM, and DATA blocks of HFiles belonging to this
101    * family will be loaded into the cache as soon as the file is opened. These
102    * loads will not count as cache misses.
103    */
104   public static final String PREFETCH_BLOCKS_ON_OPEN = "PREFETCH_BLOCKS_ON_OPEN";
105 
106   /**
107    * Size of storefile/hfile 'blocks'.  Default is {@link #DEFAULT_BLOCKSIZE}.
108    * Use smaller block sizes for faster random-access at expense of larger
109    * indices (more memory consumption).
110    */
111   public static final String BLOCKSIZE = "BLOCKSIZE";
112 
113   public static final String LENGTH = "LENGTH";
114   public static final String TTL = "TTL";
115   public static final String BLOOMFILTER = "BLOOMFILTER";
116   public static final String FOREVER = "FOREVER";
117   public static final String REPLICATION_SCOPE = "REPLICATION_SCOPE";
118   public static final byte[] REPLICATION_SCOPE_BYTES = Bytes.toBytes(REPLICATION_SCOPE);
119   public static final String MIN_VERSIONS = "MIN_VERSIONS";
120   public static final String KEEP_DELETED_CELLS = "KEEP_DELETED_CELLS";
121   public static final String COMPRESS_TAGS = "COMPRESS_TAGS";
122 
123   public static final String ENCRYPTION = "ENCRYPTION";
124   public static final String ENCRYPTION_KEY = "ENCRYPTION_KEY";
125 
126   /**
127    * Default compression type.
128    */
129   public static final String DEFAULT_COMPRESSION =
130     Compression.Algorithm.NONE.getName();
131 
132   /**
133    * Default value of the flag that enables data block encoding on disk, as
134    * opposed to encoding in cache only. We encode blocks everywhere by default,
135    * as long as {@link #DATA_BLOCK_ENCODING} is not NONE.
136    */
137   public static final boolean DEFAULT_ENCODE_ON_DISK = true;
138 
139   /** Default data block encoding algorithm. */
140   public static final String DEFAULT_DATA_BLOCK_ENCODING =
141       DataBlockEncoding.NONE.toString();
142 
143   /**
144    * Default number of versions of a record to keep.
145    */
146   public static final int DEFAULT_VERSIONS = HBaseConfiguration.create().getInt(
147     "hbase.column.max.version", 1);
148 
149   /**
150    * Default is not to keep a minimum of versions.
151    */
152   public static final int DEFAULT_MIN_VERSIONS = 0;
153 
154   /*
155    * Cache here the HCD value.
156    * Question: its OK to cache since when we're reenable, we create a new HCD?
157    */
158   private volatile Integer blocksize = null;
159 
160   /**
161    * Default setting for whether to try and serve this column family from memory or not.
162    */
163   public static final boolean DEFAULT_IN_MEMORY = false;
164 
165   /**
166    * Default setting for preventing deleted from being collected immediately.
167    */
168   public static final KeepDeletedCells DEFAULT_KEEP_DELETED = KeepDeletedCells.FALSE;
169 
170   /**
171    * Default setting for whether to use a block cache or not.
172    */
173   public static final boolean DEFAULT_BLOCKCACHE = true;
174 
175   /**
176    * Default setting for whether to cache data blocks on write if block caching
177    * is enabled.
178    */
179   public static final boolean DEFAULT_CACHE_DATA_ON_WRITE = false;
180 
181   /**
182    * Default setting for whether to cache data blocks in L1 tier.  Only makes sense if more than
183    * one tier in operations: i.e. if we have an L1 and a L2.  This will be the cases if we are
184    * using BucketCache.
185    */
186   public static final boolean DEFAULT_CACHE_DATA_IN_L1 = false;
187 
188   /**
189    * Default setting for whether to cache index blocks on write if block
190    * caching is enabled.
191    */
192   public static final boolean DEFAULT_CACHE_INDEX_ON_WRITE = false;
193 
194   /**
195    * Default size of blocks in files stored to the filesytem (hfiles).
196    */
197   public static final int DEFAULT_BLOCKSIZE = HConstants.DEFAULT_BLOCKSIZE;
198 
199   /**
200    * Default setting for whether or not to use bloomfilters.
201    */
202   public static final String DEFAULT_BLOOMFILTER = BloomType.ROW.toString();
203 
204   /**
205    * Default setting for whether to cache bloom filter blocks on write if block
206    * caching is enabled.
207    */
208   public static final boolean DEFAULT_CACHE_BLOOMS_ON_WRITE = false;
209 
210   /**
211    * Default time to live of cell contents.
212    */
213   public static final int DEFAULT_TTL = HConstants.FOREVER;
214 
215   /**
216    * Default scope.
217    */
218   public static final int DEFAULT_REPLICATION_SCOPE = HConstants.REPLICATION_SCOPE_LOCAL;
219 
220   /**
221    * Default setting for whether to evict cached blocks from the blockcache on
222    * close.
223    */
224   public static final boolean DEFAULT_EVICT_BLOCKS_ON_CLOSE = false;
225 
226   /**
227    * Default compress tags along with any type of DataBlockEncoding.
228    */
229   public static final boolean DEFAULT_COMPRESS_TAGS = true;
230 
231   /*
232    * Default setting for whether to prefetch blocks into the blockcache on open.
233    */
234   public static final boolean DEFAULT_PREFETCH_BLOCKS_ON_OPEN = false;
235 
236   private final static Map<String, String> DEFAULT_VALUES
237     = new HashMap<String, String>();
238   private final static Set<ImmutableBytesWritable> RESERVED_KEYWORDS
239     = new HashSet<ImmutableBytesWritable>();
240   static {
241       DEFAULT_VALUES.put(BLOOMFILTER, DEFAULT_BLOOMFILTER);
242       DEFAULT_VALUES.put(REPLICATION_SCOPE, String.valueOf(DEFAULT_REPLICATION_SCOPE));
243       DEFAULT_VALUES.put(HConstants.VERSIONS, String.valueOf(DEFAULT_VERSIONS));
244       DEFAULT_VALUES.put(MIN_VERSIONS, String.valueOf(DEFAULT_MIN_VERSIONS));
245       DEFAULT_VALUES.put(COMPRESSION, DEFAULT_COMPRESSION);
246       DEFAULT_VALUES.put(TTL, String.valueOf(DEFAULT_TTL));
247       DEFAULT_VALUES.put(BLOCKSIZE, String.valueOf(DEFAULT_BLOCKSIZE));
248       DEFAULT_VALUES.put(HConstants.IN_MEMORY, String.valueOf(DEFAULT_IN_MEMORY));
249       DEFAULT_VALUES.put(BLOCKCACHE, String.valueOf(DEFAULT_BLOCKCACHE));
250       DEFAULT_VALUES.put(KEEP_DELETED_CELLS, String.valueOf(DEFAULT_KEEP_DELETED));
251       DEFAULT_VALUES.put(DATA_BLOCK_ENCODING, String.valueOf(DEFAULT_DATA_BLOCK_ENCODING));
252       DEFAULT_VALUES.put(CACHE_DATA_ON_WRITE, String.valueOf(DEFAULT_CACHE_DATA_ON_WRITE));
253       DEFAULT_VALUES.put(CACHE_DATA_IN_L1, String.valueOf(DEFAULT_CACHE_DATA_IN_L1));
254       DEFAULT_VALUES.put(CACHE_INDEX_ON_WRITE, String.valueOf(DEFAULT_CACHE_INDEX_ON_WRITE));
255       DEFAULT_VALUES.put(CACHE_BLOOMS_ON_WRITE, String.valueOf(DEFAULT_CACHE_BLOOMS_ON_WRITE));
256       DEFAULT_VALUES.put(EVICT_BLOCKS_ON_CLOSE, String.valueOf(DEFAULT_EVICT_BLOCKS_ON_CLOSE));
257       DEFAULT_VALUES.put(PREFETCH_BLOCKS_ON_OPEN, String.valueOf(DEFAULT_PREFETCH_BLOCKS_ON_OPEN));
258       for (String s : DEFAULT_VALUES.keySet()) {
259         RESERVED_KEYWORDS.add(new ImmutableBytesWritable(Bytes.toBytes(s)));
260       }
261       RESERVED_KEYWORDS.add(new ImmutableBytesWritable(Bytes.toBytes(ENCRYPTION)));
262       RESERVED_KEYWORDS.add(new ImmutableBytesWritable(Bytes.toBytes(ENCRYPTION_KEY)));
263   }
264 
265   private static final int UNINITIALIZED = -1;
266 
267   // Column family name
268   private byte [] name;
269 
270   // Column metadata
271   private final Map<ImmutableBytesWritable, ImmutableBytesWritable> values =
272     new HashMap<ImmutableBytesWritable,ImmutableBytesWritable>();
273 
274   /**
275    * A map which holds the configuration specific to the column family.
276    * The keys of the map have the same names as config keys and override the defaults with
277    * cf-specific settings. Example usage may be for compactions, etc.
278    */
279   private final Map<String, String> configuration = new HashMap<String, String>();
280 
281   /*
282    * Cache the max versions rather than calculate it every time.
283    */
284   private int cachedMaxVersions = UNINITIALIZED;
285 
286   /**
287    * Default constructor.
288    * @deprecated As of release 0.96
289    *             (<a href="https://issues.apache.org/jira/browse/HBASE-5453">HBASE-5453</a>).
290    *             This will be made private in HBase 2.0.0.
291    *             Used by Writables and Writables are going away.
292    */
293   @Deprecated
294   // Make this private rather than remove after deprecation period elapses.  Its needed by pb
295   // deserializations.
296   public HColumnDescriptor() {
297     this.name = null;
298   }
299 
300   /**
301    * Construct a column descriptor specifying only the family name
302    * The other attributes are defaulted.
303    *
304    * @param familyName Column family name. Must be 'printable' -- digit or
305    * letter -- and may not contain a <code>:<code>
306    */
307   public HColumnDescriptor(final String familyName) {
308     this(Bytes.toBytes(familyName));
309   }
310 
311   /**
312    * Construct a column descriptor specifying only the family name
313    * The other attributes are defaulted.
314    *
315    * @param familyName Column family name. Must be 'printable' -- digit or
316    * letter -- and may not contain a <code>:<code>
317    */
318   public HColumnDescriptor(final byte [] familyName) {
319     this (familyName == null || familyName.length <= 0?
320       HConstants.EMPTY_BYTE_ARRAY: familyName, DEFAULT_VERSIONS,
321       DEFAULT_COMPRESSION, DEFAULT_IN_MEMORY, DEFAULT_BLOCKCACHE,
322       DEFAULT_TTL, DEFAULT_BLOOMFILTER);
323   }
324 
325   /**
326    * Constructor.
327    * Makes a deep copy of the supplied descriptor.
328    * Can make a modifiable descriptor from an UnmodifyableHColumnDescriptor.
329    * @param desc The descriptor.
330    */
331   public HColumnDescriptor(HColumnDescriptor desc) {
332     super();
333     this.name = desc.name.clone();
334     for (Map.Entry<ImmutableBytesWritable, ImmutableBytesWritable> e:
335         desc.values.entrySet()) {
336       this.values.put(e.getKey(), e.getValue());
337     }
338     for (Map.Entry<String, String> e : desc.configuration.entrySet()) {
339       this.configuration.put(e.getKey(), e.getValue());
340     }
341     setMaxVersions(desc.getMaxVersions());
342   }
343 
344   /**
345    * Constructor
346    * @param familyName Column family name. Must be 'printable' -- digit or
347    * letter -- and may not contain a <code>:<code>
348    * @param maxVersions Maximum number of versions to keep
349    * @param compression Compression type
350    * @param inMemory If true, column data should be kept in an HRegionServer's
351    * cache
352    * @param blockCacheEnabled If true, MapFile blocks should be cached
353    * @param timeToLive Time-to-live of cell contents, in seconds
354    * (use HConstants.FOREVER for unlimited TTL)
355    * @param bloomFilter Bloom filter type for this column
356    *
357    * @throws IllegalArgumentException if passed a family name that is made of
358    * other than 'word' characters: i.e. <code>[a-zA-Z_0-9]</code> or contains
359    * a <code>:</code>
360    * @throws IllegalArgumentException if the number of versions is &lt;= 0
361    * @deprecated As of release 0.96
362    *             (<a href="https://issues.apache.org/jira/browse/HBASE-">HBASE-</a>).
363    *             This will be removed in HBase 2.0.0.
364    *             Use {@link #HColumnDescriptor(String)} and setters.
365    */
366   @Deprecated
367   public HColumnDescriptor(final byte [] familyName, final int maxVersions,
368       final String compression, final boolean inMemory,
369       final boolean blockCacheEnabled,
370       final int timeToLive, final String bloomFilter) {
371     this(familyName, maxVersions, compression, inMemory, blockCacheEnabled,
372       DEFAULT_BLOCKSIZE, timeToLive, bloomFilter, DEFAULT_REPLICATION_SCOPE);
373   }
374 
375   /**
376    * Constructor
377    * @param familyName Column family name. Must be 'printable' -- digit or
378    * letter -- and may not contain a <code>:<code>
379    * @param maxVersions Maximum number of versions to keep
380    * @param compression Compression type
381    * @param inMemory If true, column data should be kept in an HRegionServer's
382    * cache
383    * @param blockCacheEnabled If true, MapFile blocks should be cached
384    * @param blocksize Block size to use when writing out storefiles.  Use
385    * smaller block sizes for faster random-access at expense of larger indices
386    * (more memory consumption).  Default is usually 64k.
387    * @param timeToLive Time-to-live of cell contents, in seconds
388    * (use HConstants.FOREVER for unlimited TTL)
389    * @param bloomFilter Bloom filter type for this column
390    * @param scope The scope tag for this column
391    *
392    * @throws IllegalArgumentException if passed a family name that is made of
393    * other than 'word' characters: i.e. <code>[a-zA-Z_0-9]</code> or contains
394    * a <code>:</code>
395    * @throws IllegalArgumentException if the number of versions is &lt;= 0
396    * @deprecated As of release 0.96
397    *             (<a href="https://issues.apache.org/jira/browse/HBASE-">HBASE-</a>).
398    *             This will be removed in HBase 2.0.0.
399    *             Use {@link #HColumnDescriptor(String)} and setters.
400    */
401   @Deprecated
402   public HColumnDescriptor(final byte [] familyName, final int maxVersions,
403       final String compression, final boolean inMemory,
404       final boolean blockCacheEnabled, final int blocksize,
405       final int timeToLive, final String bloomFilter, final int scope) {
406     this(familyName, DEFAULT_MIN_VERSIONS, maxVersions, DEFAULT_KEEP_DELETED,
407         compression, DEFAULT_ENCODE_ON_DISK, DEFAULT_DATA_BLOCK_ENCODING,
408         inMemory, blockCacheEnabled, blocksize, timeToLive, bloomFilter,
409         scope);
410   }
411 
412   /**
413    * Constructor
414    * @param familyName Column family name. Must be 'printable' -- digit or
415    * letter -- and may not contain a <code>:<code>
416    * @param minVersions Minimum number of versions to keep
417    * @param maxVersions Maximum number of versions to keep
418    * @param keepDeletedCells Whether to retain deleted cells until they expire
419    *        up to maxVersions versions.
420    * @param compression Compression type
421    * @param encodeOnDisk whether to use the specified data block encoding
422    *        on disk. If false, the encoding will be used in cache only.
423    * @param dataBlockEncoding data block encoding
424    * @param inMemory If true, column data should be kept in an HRegionServer's
425    * cache
426    * @param blockCacheEnabled If true, MapFile blocks should be cached
427    * @param blocksize Block size to use when writing out storefiles.  Use
428    * smaller blocksizes for faster random-access at expense of larger indices
429    * (more memory consumption).  Default is usually 64k.
430    * @param timeToLive Time-to-live of cell contents, in seconds
431    * (use HConstants.FOREVER for unlimited TTL)
432    * @param bloomFilter Bloom filter type for this column
433    * @param scope The scope tag for this column
434    *
435    * @throws IllegalArgumentException if passed a family name that is made of
436    * other than 'word' characters: i.e. <code>[a-zA-Z_0-9]</code> or contains
437    * a <code>:</code>
438    * @throws IllegalArgumentException if the number of versions is &lt;= 0
439    * @deprecated As of release 0.96
440    *             (<a href="https://issues.apache.org/jira/browse/HBASE-">HBASE-</a>).
441    *             This will be removed in HBase 2.0.0.
442    *             Use {@link #HColumnDescriptor(String)} and setters.
443    */
444   @Deprecated
445   public HColumnDescriptor(final byte[] familyName, final int minVersions,
446       final int maxVersions, final KeepDeletedCells keepDeletedCells,
447       final String compression, final boolean encodeOnDisk,
448       final String dataBlockEncoding, final boolean inMemory,
449       final boolean blockCacheEnabled, final int blocksize,
450       final int timeToLive, final String bloomFilter, final int scope) {
451     isLegalFamilyName(familyName);
452     this.name = familyName;
453 
454     if (maxVersions <= 0) {
455       // TODO: Allow maxVersion of 0 to be the way you say "Keep all versions".
456       // Until there is support, consider 0 or < 0 -- a configuration error.
457       throw new IllegalArgumentException("Maximum versions must be positive");
458     }
459 
460     if (minVersions > 0) {
461       if (timeToLive == HConstants.FOREVER) {
462         throw new IllegalArgumentException("Minimum versions requires TTL.");
463       }
464       if (minVersions >= maxVersions) {
465         throw new IllegalArgumentException("Minimum versions must be < "
466             + "maximum versions.");
467       }
468     }
469 
470     setMaxVersions(maxVersions);
471     setMinVersions(minVersions);
472     setKeepDeletedCells(keepDeletedCells);
473     setInMemory(inMemory);
474     setBlockCacheEnabled(blockCacheEnabled);
475     setTimeToLive(timeToLive);
476     setCompressionType(Compression.Algorithm.
477       valueOf(compression.toUpperCase()));
478     setDataBlockEncoding(DataBlockEncoding.
479         valueOf(dataBlockEncoding.toUpperCase()));
480     setBloomFilterType(BloomType.
481       valueOf(bloomFilter.toUpperCase()));
482     setBlocksize(blocksize);
483     setScope(scope);
484   }
485 
486   /**
487    * @param b Family name.
488    * @return <code>b</code>
489    * @throws IllegalArgumentException If not null and not a legitimate family
490    * name: i.e. 'printable' and ends in a ':' (Null passes are allowed because
491    * <code>b</code> can be null when deserializing).  Cannot start with a '.'
492    * either. Also Family can not be an empty value or equal "recovered.edits".
493    */
494   public static byte [] isLegalFamilyName(final byte [] b) {
495     if (b == null) {
496       return b;
497     }
498     Preconditions.checkArgument(b.length != 0, "Family name can not be empty");
499     if (b[0] == '.') {
500       throw new IllegalArgumentException("Family names cannot start with a " +
501         "period: " + Bytes.toString(b));
502     }
503     for (int i = 0; i < b.length; i++) {
504       if (Character.isISOControl(b[i]) || b[i] == ':' || b[i] == '\\' || b[i] == '/') {
505         throw new IllegalArgumentException("Illegal character <" + b[i] +
506           ">. Family names cannot contain control characters or colons: " +
507           Bytes.toString(b));
508       }
509     }
510     byte[] recoveredEdit = Bytes.toBytes(HConstants.RECOVERED_EDITS_DIR);
511     if (Bytes.equals(recoveredEdit, b)) {
512       throw new IllegalArgumentException("Family name cannot be: " +
513           HConstants.RECOVERED_EDITS_DIR);
514     }
515     return b;
516   }
517 
518   /**
519    * @return Name of this column family
520    */
521   public byte [] getName() {
522     return name;
523   }
524 
525   /**
526    * @return Name of this column family
527    */
528   public String getNameAsString() {
529     return Bytes.toString(this.name);
530   }
531 
532   /**
533    * @param key The key.
534    * @return The value.
535    */
536   public byte[] getValue(byte[] key) {
537     ImmutableBytesWritable ibw = values.get(new ImmutableBytesWritable(key));
538     if (ibw == null)
539       return null;
540     return ibw.get();
541   }
542 
543   /**
544    * @param key The key.
545    * @return The value as a string.
546    */
547   public String getValue(String key) {
548     byte[] value = getValue(Bytes.toBytes(key));
549     if (value == null)
550       return null;
551     return Bytes.toString(value);
552   }
553 
554   /**
555    * @return All values.
556    */
557   public Map<ImmutableBytesWritable,ImmutableBytesWritable> getValues() {
558     // shallow pointer copy
559     return Collections.unmodifiableMap(values);
560   }
561 
562   /**
563    * @param key The key.
564    * @param value The value.
565    * @return this (for chained invocation)
566    */
567   public HColumnDescriptor setValue(byte[] key, byte[] value) {
568     values.put(new ImmutableBytesWritable(key),
569       new ImmutableBytesWritable(value));
570     return this;
571   }
572 
573   /**
574    * @param key Key whose key and value we're to remove from HCD parameters.
575    */
576   public void remove(final byte [] key) {
577     values.remove(new ImmutableBytesWritable(key));
578   }
579 
580   /**
581    * @param key The key.
582    * @param value The value.
583    * @return this (for chained invocation)
584    */
585   public HColumnDescriptor setValue(String key, String value) {
586     if (value == null) {
587       remove(Bytes.toBytes(key));
588     } else {
589       setValue(Bytes.toBytes(key), Bytes.toBytes(value));
590     }
591     return this;
592   }
593 
594   /** @return compression type being used for the column family */
595   public Compression.Algorithm getCompression() {
596     String n = getValue(COMPRESSION);
597     if (n == null) {
598       return Compression.Algorithm.NONE;
599     }
600     return Compression.Algorithm.valueOf(n.toUpperCase());
601   }
602 
603   /** @return compression type being used for the column family for major
604       compression */
605   public Compression.Algorithm getCompactionCompression() {
606     String n = getValue(COMPRESSION_COMPACT);
607     if (n == null) {
608       return getCompression();
609     }
610     return Compression.Algorithm.valueOf(n.toUpperCase());
611   }
612 
613   /** @return maximum number of versions */
614   public int getMaxVersions() {
615     if (this.cachedMaxVersions == UNINITIALIZED) {
616       String v = getValue(HConstants.VERSIONS);
617       this.cachedMaxVersions = Integer.parseInt(v);
618     }
619     return this.cachedMaxVersions;
620   }
621 
622   /**
623    * @param maxVersions maximum number of versions
624    * @return this (for chained invocation)
625    */
626   public HColumnDescriptor setMaxVersions(int maxVersions) {
627     if (maxVersions <= 0) {
628       // TODO: Allow maxVersion of 0 to be the way you say "Keep all versions".
629       // Until there is support, consider 0 or < 0 -- a configuration error.
630       throw new IllegalArgumentException("Maximum versions must be positive");
631     }
632     if (maxVersions < this.getMinVersions()) {
633         throw new IllegalArgumentException("Set MaxVersion to " + maxVersions
634             + " while minVersion is " + this.getMinVersions()
635             + ". Maximum versions must be >= minimum versions ");
636     }
637     setValue(HConstants.VERSIONS, Integer.toString(maxVersions));
638     cachedMaxVersions = maxVersions;
639     return this;
640   }
641 
642   /**
643    * @return The storefile/hfile blocksize for this column family.
644    */
645   public synchronized int getBlocksize() {
646     if (this.blocksize == null) {
647       String value = getValue(BLOCKSIZE);
648       this.blocksize = (value != null)?
649         Integer.decode(value): Integer.valueOf(DEFAULT_BLOCKSIZE);
650     }
651     return this.blocksize.intValue();
652   }
653 
654   /**
655    * @param s Blocksize to use when writing out storefiles/hfiles on this
656    * column family.
657    * @return this (for chained invocation)
658    */
659   public HColumnDescriptor setBlocksize(int s) {
660     setValue(BLOCKSIZE, Integer.toString(s));
661     this.blocksize = null;
662     return this;
663   }
664 
665   /**
666    * @return Compression type setting.
667    */
668   public Compression.Algorithm getCompressionType() {
669     return getCompression();
670   }
671 
672   /**
673    * Compression types supported in hbase.
674    * LZO is not bundled as part of the hbase distribution.
675    * See <a href="http://wiki.apache.org/hadoop/UsingLzoCompression">LZO Compression</a>
676    * for how to enable it.
677    * @param type Compression type setting.
678    * @return this (for chained invocation)
679    */
680   public HColumnDescriptor setCompressionType(Compression.Algorithm type) {
681     return setValue(COMPRESSION, type.getName().toUpperCase());
682   }
683 
684   /**
685    * @return data block encoding algorithm used on disk
686    * @deprecated As of release 0.98
687    *             (<a href="https://issues.apache.org/jira/browse/HBASE-9870">HBASE-9870</a>).
688    *             This will be removed in HBase 2.0.0. See {@link #getDataBlockEncoding()}}
689    */
690   @Deprecated
691   public DataBlockEncoding getDataBlockEncodingOnDisk() {
692     return getDataBlockEncoding();
693   }
694 
695   /**
696    * This method does nothing now. Flag ENCODE_ON_DISK is not used
697    * any more. Data blocks have the same encoding in cache as on disk.
698    * @return this (for chained invocation)
699    * @deprecated As of release 0.98
700    *             (<a href="https://issues.apache.org/jira/browse/HBASE-9870">HBASE-9870</a>).
701    *             This will be removed in HBase 2.0.0. This method does nothing now.
702    */
703   @Deprecated
704   public HColumnDescriptor setEncodeOnDisk(boolean encodeOnDisk) {
705     return this;
706   }
707 
708   /**
709    * @return the data block encoding algorithm used in block cache and
710    *         optionally on disk
711    */
712   public DataBlockEncoding getDataBlockEncoding() {
713     String type = getValue(DATA_BLOCK_ENCODING);
714     if (type == null) {
715       type = DEFAULT_DATA_BLOCK_ENCODING;
716     }
717     return DataBlockEncoding.valueOf(type);
718   }
719 
720   /**
721    * Set data block encoding algorithm used in block cache.
722    * @param type What kind of data block encoding will be used.
723    * @return this (for chained invocation)
724    */
725   public HColumnDescriptor setDataBlockEncoding(DataBlockEncoding type) {
726     String name;
727     if (type != null) {
728       name = type.toString();
729     } else {
730       name = DataBlockEncoding.NONE.toString();
731     }
732     return setValue(DATA_BLOCK_ENCODING, name);
733   }
734 
735   /**
736    * Set whether the tags should be compressed along with DataBlockEncoding. When no
737    * DataBlockEncoding is been used, this is having no effect.
738    *
739    * @param compressTags
740    * @return this (for chained invocation)
741    */
742   public HColumnDescriptor setCompressTags(boolean compressTags) {
743     return setValue(COMPRESS_TAGS, String.valueOf(compressTags));
744   }
745 
746   /**
747    * @return Whether KV tags should be compressed along with DataBlockEncoding. When no
748    *         DataBlockEncoding is been used, this is having no effect.
749    * @deprecated As of release 1.0.0
750    *             (<a href="https://issues.apache.org/jira/browse/HBASE-10870">HBASE-10870</a>).
751    *             This will be removed in HBase 2.0.0. Use {@link #isCompressTags()} instead.
752    */
753   @Deprecated
754   public boolean shouldCompressTags() {
755     String compressTagsStr = getValue(COMPRESS_TAGS);
756     boolean compressTags = DEFAULT_COMPRESS_TAGS;
757     if (compressTagsStr != null) {
758       compressTags = Boolean.valueOf(compressTagsStr);
759     }
760     return compressTags;
761   }
762 
763   /**
764    * @return Whether KV tags should be compressed along with DataBlockEncoding. When no
765    *         DataBlockEncoding is been used, this is having no effect.
766    */
767   public boolean isCompressTags() {
768     String compressTagsStr = getValue(COMPRESS_TAGS);
769     boolean compressTags = DEFAULT_COMPRESS_TAGS;
770     if (compressTagsStr != null) {
771       compressTags = Boolean.valueOf(compressTagsStr);
772     }
773     return compressTags;
774   }
775 
776   /**
777    * @return Compression type setting.
778    */
779   public Compression.Algorithm getCompactionCompressionType() {
780     return getCompactionCompression();
781   }
782 
783   /**
784    * Compression types supported in hbase.
785    * LZO is not bundled as part of the hbase distribution.
786    * See <a href="http://wiki.apache.org/hadoop/UsingLzoCompression">LZO Compression</a>
787    * for how to enable it.
788    * @param type Compression type setting.
789    * @return this (for chained invocation)
790    */
791   public HColumnDescriptor setCompactionCompressionType(
792       Compression.Algorithm type) {
793     return setValue(COMPRESSION_COMPACT, type.getName().toUpperCase());
794   }
795 
796   /**
797    * @return True if we are to favor keeping all values for this column family in the
798    * HRegionServer cache.
799    */
800   public boolean isInMemory() {
801     String value = getValue(HConstants.IN_MEMORY);
802     if (value != null)
803       return Boolean.valueOf(value).booleanValue();
804     return DEFAULT_IN_MEMORY;
805   }
806 
807   /**
808    * @param inMemory True if we are to favor keeping all values for this column family in the
809    * HRegionServer cache
810    * @return this (for chained invocation)
811    */
812   public HColumnDescriptor setInMemory(boolean inMemory) {
813     return setValue(HConstants.IN_MEMORY, Boolean.toString(inMemory));
814   }
815 
816   public KeepDeletedCells getKeepDeletedCells() {
817     String value = getValue(KEEP_DELETED_CELLS);
818     if (value != null) {
819       // toUpperCase for backwards compatibility
820       return KeepDeletedCells.valueOf(value.toUpperCase());
821     }
822     return DEFAULT_KEEP_DELETED;
823   }
824 
825   /**
826    * @param keepDeletedCells True if deleted rows should not be collected
827    * immediately.
828    * @return this (for chained invocation)
829    * @deprecated As of release 1.0.0
830    *             (<a href="https://issues.apache.org/jira/browse/HBASE-12363">HBASE-12363</a>).
831    *             This will be removed in HBase 2.0.0.
832    *             Use {@link #setKeepDeletedCells(KeepDeletedCells)}.
833    */
834   @Deprecated
835   public HColumnDescriptor setKeepDeletedCells(boolean keepDeletedCells) {
836     return setValue(KEEP_DELETED_CELLS, (keepDeletedCells ? KeepDeletedCells.TRUE
837         : KeepDeletedCells.FALSE).toString());
838   }
839 
840   /**
841    * @param keepDeletedCells True if deleted rows should not be collected
842    * immediately.
843    * @return this (for chained invocation)
844    */
845   public HColumnDescriptor setKeepDeletedCells(KeepDeletedCells keepDeletedCells) {
846     return setValue(KEEP_DELETED_CELLS, keepDeletedCells.toString());
847   }
848 
849   /**
850    * @return Time-to-live of cell contents, in seconds.
851    */
852   public int getTimeToLive() {
853     String value = getValue(TTL);
854     return (value != null)? Integer.valueOf(value).intValue(): DEFAULT_TTL;
855   }
856 
857   /**
858    * @param timeToLive Time-to-live of cell contents, in seconds.
859    * @return this (for chained invocation)
860    */
861   public HColumnDescriptor setTimeToLive(int timeToLive) {
862     return setValue(TTL, Integer.toString(timeToLive));
863   }
864 
865   /**
866    * @return The minimum number of versions to keep.
867    */
868   public int getMinVersions() {
869     String value = getValue(MIN_VERSIONS);
870     return (value != null)? Integer.valueOf(value).intValue(): 0;
871   }
872 
873   /**
874    * @param minVersions The minimum number of versions to keep.
875    * (used when timeToLive is set)
876    * @return this (for chained invocation)
877    */
878   public HColumnDescriptor setMinVersions(int minVersions) {
879     return setValue(MIN_VERSIONS, Integer.toString(minVersions));
880   }
881 
882   /**
883    * @return True if hfile DATA type blocks should be cached (You cannot disable caching of INDEX
884    * and BLOOM type blocks).
885    */
886   public boolean isBlockCacheEnabled() {
887     String value = getValue(BLOCKCACHE);
888     if (value != null)
889       return Boolean.valueOf(value).booleanValue();
890     return DEFAULT_BLOCKCACHE;
891   }
892 
893   /**
894    * @param blockCacheEnabled True if hfile DATA type blocks should be cached (We always cache
895    * INDEX and BLOOM blocks; you cannot turn this off).
896    * @return this (for chained invocation)
897    */
898   public HColumnDescriptor setBlockCacheEnabled(boolean blockCacheEnabled) {
899     return setValue(BLOCKCACHE, Boolean.toString(blockCacheEnabled));
900   }
901 
902   /**
903    * @return bloom filter type used for new StoreFiles in ColumnFamily
904    */
905   public BloomType getBloomFilterType() {
906     String n = getValue(BLOOMFILTER);
907     if (n == null) {
908       n = DEFAULT_BLOOMFILTER;
909     }
910     return BloomType.valueOf(n.toUpperCase());
911   }
912 
913   /**
914    * @param bt bloom filter type
915    * @return this (for chained invocation)
916    */
917   public HColumnDescriptor setBloomFilterType(final BloomType bt) {
918     return setValue(BLOOMFILTER, bt.toString());
919   }
920 
921    /**
922     * @return the scope tag
923     */
924   public int getScope() {
925     byte[] value = getValue(REPLICATION_SCOPE_BYTES);
926     if (value != null) {
927       return Integer.valueOf(Bytes.toString(value));
928     }
929     return DEFAULT_REPLICATION_SCOPE;
930   }
931 
932  /**
933   * @param scope the scope tag
934   * @return this (for chained invocation)
935   */
936   public HColumnDescriptor setScope(int scope) {
937     return setValue(REPLICATION_SCOPE, Integer.toString(scope));
938   }
939 
940   /**
941    * @return true if we should cache data blocks on write
942    * @deprecated As of release 1.0.0
943    *             (<a href="https://issues.apache.org/jira/browse/HBASE-10870">HBASE-10870</a>).
944    *             This will be removed in HBase 2.0.0. Use {@link #isCacheDataOnWrite()}} instead.
945    */
946   @Deprecated
947   public boolean shouldCacheDataOnWrite() {
948     return setAndGetBoolean(CACHE_DATA_ON_WRITE, DEFAULT_CACHE_DATA_ON_WRITE);
949   }
950 
951   /**
952    * @return true if we should cache data blocks on write
953    */
954   public boolean isCacheDataOnWrite() {
955     return setAndGetBoolean(CACHE_DATA_ON_WRITE, DEFAULT_CACHE_DATA_ON_WRITE);
956   }
957 
958   /**
959    * @param value true if we should cache data blocks on write
960    * @return this (for chained invocation)
961    */
962   public HColumnDescriptor setCacheDataOnWrite(boolean value) {
963     return setValue(CACHE_DATA_ON_WRITE, Boolean.toString(value));
964   }
965 
966   /**
967    * @return true if we should cache data blocks in the L1 cache (if block cache deploy
968    * has more than one tier; e.g. we are using CombinedBlockCache).
969    * @deprecated As of release 1.0.0
970    *             (<a href="https://issues.apache.org/jira/browse/HBASE-10870">HBASE-10870</a>).
971    *             This will be removed in HBase 2.0.0. Use {@link #isCacheDataInL1()}} instead.
972    */
973   @Deprecated
974   public boolean shouldCacheDataInL1() {
975     return setAndGetBoolean(CACHE_DATA_IN_L1, DEFAULT_CACHE_DATA_IN_L1);
976   }
977 
978   /**
979    * @return true if we should cache data blocks in the L1 cache (if block cache deploy has more
980    *         than one tier; e.g. we are using CombinedBlockCache).
981    */
982   public boolean isCacheDataInL1() {
983     return setAndGetBoolean(CACHE_DATA_IN_L1, DEFAULT_CACHE_DATA_IN_L1);
984   }
985 
986   /**
987    * @param value true if we should cache data blocks in the L1 cache (if block cache deploy
988    * has more than one tier; e.g. we are using CombinedBlockCache).
989    * @return this (for chained invocation)
990    */
991   public HColumnDescriptor setCacheDataInL1(boolean value) {
992     return setValue(CACHE_DATA_IN_L1, Boolean.toString(value));
993   }
994 
995   private boolean setAndGetBoolean(final String key, final boolean defaultSetting) {
996     String value = getValue(key);
997     if (value != null) return Boolean.valueOf(value).booleanValue();
998     return defaultSetting;
999   }
1000 
1001   /**
1002    * @return true if we should cache index blocks on write
1003    * @deprecated As of release 1.0.0
1004    *             (<a href="https://issues.apache.org/jira/browse/HBASE-10870">HBASE-10870</a>).
1005    *             This will be removed in HBase 2.0.0.
1006    *             Use {@link #isCacheIndexesOnWrite()} instead.
1007    */
1008   @Deprecated
1009   public boolean shouldCacheIndexesOnWrite() {
1010     return setAndGetBoolean(CACHE_INDEX_ON_WRITE, DEFAULT_CACHE_INDEX_ON_WRITE);
1011   }
1012 
1013   /**
1014    * @return true if we should cache index blocks on write
1015    */
1016   public boolean isCacheIndexesOnWrite() {
1017     return setAndGetBoolean(CACHE_INDEX_ON_WRITE, DEFAULT_CACHE_INDEX_ON_WRITE);
1018   }
1019 
1020   /**
1021    * @param value true if we should cache index blocks on write
1022    * @return this (for chained invocation)
1023    */
1024   public HColumnDescriptor setCacheIndexesOnWrite(boolean value) {
1025     return setValue(CACHE_INDEX_ON_WRITE, Boolean.toString(value));
1026   }
1027 
1028   /**
1029    * @return true if we should cache bloomfilter blocks on write
1030    * @deprecated As of release 1.0.0
1031    *             (<a href="https://issues.apache.org/jira/browse/HBASE-10870">HBASE-10870</a>).
1032    *             This will be removed in HBase 2.0.0.
1033    *             Use {@link #isCacheBloomsOnWrite()}} instead.
1034    */
1035   @Deprecated
1036   public boolean shouldCacheBloomsOnWrite() {
1037     return setAndGetBoolean(CACHE_BLOOMS_ON_WRITE, DEFAULT_CACHE_BLOOMS_ON_WRITE);
1038   }
1039 
1040   /**
1041    * @return true if we should cache bloomfilter blocks on write
1042    */
1043   public boolean isCacheBloomsOnWrite() {
1044     return setAndGetBoolean(CACHE_BLOOMS_ON_WRITE, DEFAULT_CACHE_BLOOMS_ON_WRITE);
1045   }
1046 
1047   /**
1048    * @param value true if we should cache bloomfilter blocks on write
1049    * @return this (for chained invocation)
1050    */
1051   public HColumnDescriptor setCacheBloomsOnWrite(boolean value) {
1052     return setValue(CACHE_BLOOMS_ON_WRITE, Boolean.toString(value));
1053   }
1054 
1055   /**
1056    * @return true if we should evict cached blocks from the blockcache on
1057    * close
1058    * @deprecated As of release 1.0.0
1059    *             (<a href="https://issues.apache.org/jira/browse/HBASE-10870">HBASE-10870</a>).
1060    *             This will be removed in HBase 2.0.0.
1061    *             Use {@link #isEvictBlocksOnClose()}} instead.
1062    */
1063   @Deprecated
1064   public boolean shouldEvictBlocksOnClose() {
1065     return setAndGetBoolean(EVICT_BLOCKS_ON_CLOSE, DEFAULT_EVICT_BLOCKS_ON_CLOSE);
1066   }
1067 
1068   /**
1069    * @return true if we should evict cached blocks from the blockcache on close
1070    */
1071   public boolean isEvictBlocksOnClose() {
1072     return setAndGetBoolean(EVICT_BLOCKS_ON_CLOSE, DEFAULT_EVICT_BLOCKS_ON_CLOSE);
1073   }
1074 
1075   /**
1076    * @param value true if we should evict cached blocks from the blockcache on
1077    * close
1078    * @return this (for chained invocation)
1079    */
1080   public HColumnDescriptor setEvictBlocksOnClose(boolean value) {
1081     return setValue(EVICT_BLOCKS_ON_CLOSE, Boolean.toString(value));
1082   }
1083 
1084   /**
1085    * @return true if we should prefetch blocks into the blockcache on open
1086    * @deprecated As of release 1.0.0
1087    *             (<a href="https://issues.apache.org/jira/browse/HBASE-10870">HBASE-10870</a>).
1088    *             This will be removed in HBase 2.0.0.
1089    *             Use {@link #isPrefetchBlocksOnOpen()}}} instead.
1090    */
1091   @Deprecated
1092   public boolean shouldPrefetchBlocksOnOpen() {
1093     return setAndGetBoolean(PREFETCH_BLOCKS_ON_OPEN, DEFAULT_PREFETCH_BLOCKS_ON_OPEN);
1094   }
1095 
1096   /**
1097    * @return true if we should prefetch blocks into the blockcache on open
1098    */
1099   public boolean isPrefetchBlocksOnOpen() {
1100     return setAndGetBoolean(PREFETCH_BLOCKS_ON_OPEN, DEFAULT_PREFETCH_BLOCKS_ON_OPEN);
1101   }
1102 
1103   /**
1104    * @param value true if we should prefetch blocks into the blockcache on open
1105    * @return this (for chained invocation)
1106    */
1107   public HColumnDescriptor setPrefetchBlocksOnOpen(boolean value) {
1108     return setValue(PREFETCH_BLOCKS_ON_OPEN, Boolean.toString(value));
1109   }
1110 
1111   /**
1112    * @see java.lang.Object#toString()
1113    */
1114   @Override
1115   public String toString() {
1116     StringBuilder s = new StringBuilder();
1117 
1118     s.append('{');
1119     s.append(HConstants.NAME);
1120     s.append(" => '");
1121     s.append(Bytes.toString(name));
1122     s.append("'");
1123     s.append(getValues(true));
1124     s.append('}');
1125     return s.toString();
1126   }
1127 
1128   /**
1129    * @return Column family descriptor with only the customized attributes.
1130    */
1131   public String toStringCustomizedValues() {
1132     StringBuilder s = new StringBuilder();
1133     s.append('{');
1134     s.append(HConstants.NAME);
1135     s.append(" => '");
1136     s.append(Bytes.toString(name));
1137     s.append("'");
1138     s.append(getValues(false));
1139     s.append('}');
1140     return s.toString();
1141   }
1142 
1143   private StringBuilder getValues(boolean printDefaults) {
1144     StringBuilder s = new StringBuilder();
1145 
1146     boolean hasConfigKeys = false;
1147 
1148     // print all reserved keys first
1149     for (ImmutableBytesWritable k : values.keySet()) {
1150       if (!RESERVED_KEYWORDS.contains(k)) {
1151         hasConfigKeys = true;
1152         continue;
1153       }
1154       String key = Bytes.toString(k.get());
1155       String value = Bytes.toStringBinary(values.get(k).get());
1156       if (printDefaults
1157           || !DEFAULT_VALUES.containsKey(key)
1158           || !DEFAULT_VALUES.get(key).equalsIgnoreCase(value)) {
1159         s.append(", ");
1160         s.append(key);
1161         s.append(" => ");
1162         s.append('\'').append(PrettyPrinter.format(value, getUnit(key))).append('\'');
1163       }
1164     }
1165 
1166     // print all non-reserved, advanced config keys as a separate subset
1167     if (hasConfigKeys) {
1168       s.append(", ");
1169       s.append(HConstants.METADATA).append(" => ");
1170       s.append('{');
1171       boolean printComma = false;
1172       for (ImmutableBytesWritable k : values.keySet()) {
1173         if (RESERVED_KEYWORDS.contains(k)) {
1174           continue;
1175         }
1176         String key = Bytes.toString(k.get());
1177         String value = Bytes.toStringBinary(values.get(k).get());
1178         if (printComma) {
1179           s.append(", ");
1180         }
1181         printComma = true;
1182         s.append('\'').append(key).append('\'');
1183         s.append(" => ");
1184         s.append('\'').append(PrettyPrinter.format(value, getUnit(key))).append('\'');
1185       }
1186       s.append('}');
1187     }
1188 
1189     if (!configuration.isEmpty()) {
1190       s.append(", ");
1191       s.append(HConstants.CONFIGURATION).append(" => ");
1192       s.append('{');
1193       boolean printCommaForConfiguration = false;
1194       for (Map.Entry<String, String> e : configuration.entrySet()) {
1195         if (printCommaForConfiguration) s.append(", ");
1196         printCommaForConfiguration = true;
1197         s.append('\'').append(e.getKey()).append('\'');
1198         s.append(" => ");
1199         s.append('\'').append(PrettyPrinter.format(e.getValue(), getUnit(e.getKey()))).append('\'');
1200       }
1201       s.append("}");
1202     }
1203     return s;
1204   }
1205 
1206   public static Unit getUnit(String key) {
1207     Unit unit;
1208       /* TTL for now, we can add more as we neeed */
1209     if (key.equals(HColumnDescriptor.TTL)) {
1210       unit = Unit.TIME_INTERVAL;
1211     } else {
1212       unit = Unit.NONE;
1213     }
1214     return unit;
1215   }
1216 
1217   public static Map<String, String> getDefaultValues() {
1218     return Collections.unmodifiableMap(DEFAULT_VALUES);
1219   }
1220 
1221   /**
1222    * @see java.lang.Object#equals(java.lang.Object)
1223    */
1224   @Override
1225   public boolean equals(Object obj) {
1226     if (this == obj) {
1227       return true;
1228     }
1229     if (obj == null) {
1230       return false;
1231     }
1232     if (!(obj instanceof HColumnDescriptor)) {
1233       return false;
1234     }
1235     return compareTo((HColumnDescriptor)obj) == 0;
1236   }
1237 
1238   /**
1239    * @see java.lang.Object#hashCode()
1240    */
1241   @Override
1242   public int hashCode() {
1243     int result = Bytes.hashCode(this.name);
1244     result ^= Byte.valueOf(COLUMN_DESCRIPTOR_VERSION).hashCode();
1245     result ^= values.hashCode();
1246     result ^= configuration.hashCode();
1247     return result;
1248   }
1249 
1250   /**
1251    * @deprecated Writables are going away.  Use pb {@link #parseFrom(byte[])} instead.
1252    */
1253   @Deprecated
1254   public void readFields(DataInput in) throws IOException {
1255     int version = in.readByte();
1256     if (version < 6) {
1257       if (version <= 2) {
1258         Text t = new Text();
1259         t.readFields(in);
1260         this.name = t.getBytes();
1261 //        if(KeyValue.getFamilyDelimiterIndex(this.name, 0, this.name.length)
1262 //            > 0) {
1263 //          this.name = stripColon(this.name);
1264 //        }
1265       } else {
1266         this.name = Bytes.readByteArray(in);
1267       }
1268       this.values.clear();
1269       setMaxVersions(in.readInt());
1270       int ordinal = in.readInt();
1271       setCompressionType(Compression.Algorithm.values()[ordinal]);
1272       setInMemory(in.readBoolean());
1273       setBloomFilterType(in.readBoolean() ? BloomType.ROW : BloomType.NONE);
1274       if (getBloomFilterType() != BloomType.NONE && version < 5) {
1275         // If a bloomFilter is enabled and the column descriptor is less than
1276         // version 5, we need to skip over it to read the rest of the column
1277         // descriptor. There are no BloomFilterDescriptors written to disk for
1278         // column descriptors with a version number >= 5
1279         throw new UnsupportedClassVersionError(this.getClass().getName() +
1280             " does not support backward compatibility with versions older " +
1281             "than version 5");
1282       }
1283       if (version > 1) {
1284         setBlockCacheEnabled(in.readBoolean());
1285       }
1286       if (version > 2) {
1287        setTimeToLive(in.readInt());
1288       }
1289     } else {
1290       // version 6+
1291       this.name = Bytes.readByteArray(in);
1292       this.values.clear();
1293       int numValues = in.readInt();
1294       for (int i = 0; i < numValues; i++) {
1295         ImmutableBytesWritable key = new ImmutableBytesWritable();
1296         ImmutableBytesWritable value = new ImmutableBytesWritable();
1297         key.readFields(in);
1298         value.readFields(in);
1299 
1300         // in version 8, the BloomFilter setting changed from bool to enum
1301         if (version < 8 && Bytes.toString(key.get()).equals(BLOOMFILTER)) {
1302           value.set(Bytes.toBytes(
1303               Boolean.getBoolean(Bytes.toString(value.get()))
1304                 ? BloomType.ROW.toString()
1305                 : BloomType.NONE.toString()));
1306         }
1307 
1308         values.put(key, value);
1309       }
1310       if (version == 6) {
1311         // Convert old values.
1312         setValue(COMPRESSION, Compression.Algorithm.NONE.getName());
1313       }
1314       String value = getValue(HConstants.VERSIONS);
1315       this.cachedMaxVersions = (value != null)?
1316           Integer.valueOf(value).intValue(): DEFAULT_VERSIONS;
1317       if (version > 10) {
1318         configuration.clear();
1319         int numConfigs = in.readInt();
1320         for (int i = 0; i < numConfigs; i++) {
1321           ImmutableBytesWritable key = new ImmutableBytesWritable();
1322           ImmutableBytesWritable val = new ImmutableBytesWritable();
1323           key.readFields(in);
1324           val.readFields(in);
1325           configuration.put(
1326             Bytes.toString(key.get(), key.getOffset(), key.getLength()),
1327             Bytes.toString(val.get(), val.getOffset(), val.getLength()));
1328         }
1329       }
1330     }
1331   }
1332 
1333   /**
1334    * @deprecated Writables are going away.  Use {@link #toByteArray()} instead.
1335    */
1336   @Deprecated
1337   public void write(DataOutput out) throws IOException {
1338     out.writeByte(COLUMN_DESCRIPTOR_VERSION);
1339     Bytes.writeByteArray(out, this.name);
1340     out.writeInt(values.size());
1341     for (Map.Entry<ImmutableBytesWritable, ImmutableBytesWritable> e:
1342         values.entrySet()) {
1343       e.getKey().write(out);
1344       e.getValue().write(out);
1345     }
1346     out.writeInt(configuration.size());
1347     for (Map.Entry<String, String> e : configuration.entrySet()) {
1348       new ImmutableBytesWritable(Bytes.toBytes(e.getKey())).write(out);
1349       new ImmutableBytesWritable(Bytes.toBytes(e.getValue())).write(out);
1350     }
1351   }
1352 
1353   // Comparable
1354   @Override
1355   public int compareTo(HColumnDescriptor o) {
1356     int result = Bytes.compareTo(this.name, o.getName());
1357     if (result == 0) {
1358       // punt on comparison for ordering, just calculate difference
1359       result = this.values.hashCode() - o.values.hashCode();
1360       if (result < 0)
1361         result = -1;
1362       else if (result > 0)
1363         result = 1;
1364     }
1365     if (result == 0) {
1366       result = this.configuration.hashCode() - o.configuration.hashCode();
1367       if (result < 0)
1368         result = -1;
1369       else if (result > 0)
1370         result = 1;
1371     }
1372     return result;
1373   }
1374 
1375   /**
1376    * @return This instance serialized with pb with pb magic prefix
1377    * @see #parseFrom(byte[])
1378    */
1379   public byte [] toByteArray() {
1380     return ProtobufUtil.prependPBMagic(convert().toByteArray());
1381   }
1382 
1383   /**
1384    * @param bytes A pb serialized {@link HColumnDescriptor} instance with pb magic prefix
1385    * @return An instance of {@link HColumnDescriptor} made from <code>bytes</code>
1386    * @throws DeserializationException
1387    * @see #toByteArray()
1388    */
1389   public static HColumnDescriptor parseFrom(final byte [] bytes) throws DeserializationException {
1390     if (!ProtobufUtil.isPBMagicPrefix(bytes)) throw new DeserializationException("No magic");
1391     int pblen = ProtobufUtil.lengthOfPBMagic();
1392     ColumnFamilySchema.Builder builder = ColumnFamilySchema.newBuilder();
1393     ColumnFamilySchema cfs = null;
1394     try {
1395       ProtobufUtil.mergeFrom(builder, bytes, pblen, bytes.length - pblen);
1396       cfs = builder.build();
1397     } catch (IOException e) {
1398       throw new DeserializationException(e);
1399     }
1400     return convert(cfs);
1401   }
1402 
1403   /**
1404    * @param cfs
1405    * @return An {@link HColumnDescriptor} made from the passed in <code>cfs</code>
1406    */
1407   public static HColumnDescriptor convert(final ColumnFamilySchema cfs) {
1408     // Use the empty constructor so we preserve the initial values set on construction for things
1409     // like maxVersion.  Otherwise, we pick up wrong values on deserialization which makes for
1410     // unrelated-looking test failures that are hard to trace back to here.
1411     HColumnDescriptor hcd = new HColumnDescriptor();
1412     hcd.name = cfs.getName().toByteArray();
1413     for (BytesBytesPair a: cfs.getAttributesList()) {
1414       hcd.setValue(a.getFirst().toByteArray(), a.getSecond().toByteArray());
1415     }
1416     for (NameStringPair a: cfs.getConfigurationList()) {
1417       hcd.setConfiguration(a.getName(), a.getValue());
1418     }
1419     return hcd;
1420   }
1421 
1422   /**
1423    * @return Convert this instance to a the pb column family type
1424    */
1425   public ColumnFamilySchema convert() {
1426     ColumnFamilySchema.Builder builder = ColumnFamilySchema.newBuilder();
1427     builder.setName(ByteStringer.wrap(getName()));
1428     for (Map.Entry<ImmutableBytesWritable, ImmutableBytesWritable> e: this.values.entrySet()) {
1429       BytesBytesPair.Builder aBuilder = BytesBytesPair.newBuilder();
1430       aBuilder.setFirst(ByteStringer.wrap(e.getKey().get()));
1431       aBuilder.setSecond(ByteStringer.wrap(e.getValue().get()));
1432       builder.addAttributes(aBuilder.build());
1433     }
1434     for (Map.Entry<String, String> e : this.configuration.entrySet()) {
1435       NameStringPair.Builder aBuilder = NameStringPair.newBuilder();
1436       aBuilder.setName(e.getKey());
1437       aBuilder.setValue(e.getValue());
1438       builder.addConfiguration(aBuilder.build());
1439     }
1440     return builder.build();
1441   }
1442 
1443   /**
1444    * Getter for accessing the configuration value by key.
1445    */
1446   public String getConfigurationValue(String key) {
1447     return configuration.get(key);
1448   }
1449 
1450   /**
1451    * Getter for fetching an unmodifiable {@link #configuration} map.
1452    */
1453   public Map<String, String> getConfiguration() {
1454     // shallow pointer copy
1455     return Collections.unmodifiableMap(configuration);
1456   }
1457 
1458   /**
1459    * Setter for storing a configuration setting in {@link #configuration} map.
1460    * @param key Config key. Same as XML config key e.g. hbase.something.or.other.
1461    * @param value String value. If null, removes the configuration.
1462    */
1463   public HColumnDescriptor setConfiguration(String key, String value) {
1464     if (value == null) {
1465       removeConfiguration(key);
1466     } else {
1467       configuration.put(key, value);
1468     }
1469     return this;
1470   }
1471 
1472   /**
1473    * Remove a configuration setting represented by the key from the {@link #configuration} map.
1474    */
1475   public void removeConfiguration(final String key) {
1476     configuration.remove(key);
1477   }
1478 
1479   /**
1480    * Return the encryption algorithm in use by this family
1481    */
1482   public String getEncryptionType() {
1483     return getValue(ENCRYPTION);
1484   }
1485 
1486   /**
1487    * Set the encryption algorithm for use with this family
1488    * @param algorithm
1489    */
1490   public HColumnDescriptor setEncryptionType(String algorithm) {
1491     setValue(ENCRYPTION, algorithm);
1492     return this;
1493   }
1494 
1495   /** Return the raw crypto key attribute for the family, or null if not set  */
1496   public byte[] getEncryptionKey() {
1497     return getValue(Bytes.toBytes(ENCRYPTION_KEY));
1498   }
1499 
1500   /** Set the raw crypto key attribute for the family */
1501   public HColumnDescriptor setEncryptionKey(byte[] keyBytes) {
1502     setValue(Bytes.toBytes(ENCRYPTION_KEY), keyBytes);
1503     return this;
1504   }
1505 }