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