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