001/*
002 * Licensed to the Apache Software Foundation (ASF) under one
003 * or more contributor license agreements.  See the NOTICE file
004 * distributed with this work for additional information
005 * regarding copyright ownership.  The ASF licenses this file
006 * to you under the Apache License, Version 2.0 (the
007 * "License"); you may not use this file except in compliance
008 * with the License.  You may obtain a copy of the License at
009 *
010 *     http://www.apache.org/licenses/LICENSE-2.0
011 *
012 * Unless required by applicable law or agreed to in writing, software
013 * distributed under the License is distributed on an "AS IS" BASIS,
014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
015 * See the License for the specific language governing permissions and
016 * limitations under the License.
017 */
018package org.apache.hadoop.hbase.client;
019
020import java.io.IOException;
021import java.util.Collections;
022import java.util.HashMap;
023import java.util.HashSet;
024import java.util.Map;
025import java.util.Set;
026import java.util.function.Function;
027import org.apache.hadoop.hbase.HConstants;
028import org.apache.hadoop.hbase.KeepDeletedCells;
029import org.apache.hadoop.hbase.MemoryCompactionPolicy;
030import org.apache.hadoop.hbase.exceptions.DeserializationException;
031import org.apache.hadoop.hbase.exceptions.HBaseException;
032import org.apache.hadoop.hbase.io.compress.Compression;
033import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
034import org.apache.hadoop.hbase.io.encoding.IndexBlockEncoding;
035import org.apache.hadoop.hbase.regionserver.BloomType;
036import org.apache.hadoop.hbase.util.Bytes;
037import org.apache.hadoop.hbase.util.PrettyPrinter;
038import org.apache.hadoop.hbase.util.PrettyPrinter.Unit;
039import org.apache.yetus.audience.InterfaceAudience;
040
041import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
042
043import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
044import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ColumnFamilySchema;
045
046@InterfaceAudience.Public
047public class ColumnFamilyDescriptorBuilder {
048  // For future backward compatibility
049
050  // Version 3 was when column names become byte arrays and when we picked up
051  // Time-to-live feature. Version 4 was when we moved to byte arrays, HBASE-82.
052  // Version 5 was when bloom filter descriptors were removed.
053  // Version 6 adds metadata as a map where keys and values are byte[].
054  // Version 7 -- add new compression and hfile blocksize to HColumnDescriptor (HBASE-1217)
055  // Version 8 -- reintroduction of bloom filters, changed from boolean to enum
056  // Version 9 -- add data block encoding
057  // Version 10 -- change metadata to standard type.
058  // Version 11 -- add column family level configuration.
059  private static final byte COLUMN_DESCRIPTOR_VERSION = (byte) 11;
060
061  @InterfaceAudience.Private
062  public static final String IN_MEMORY_COMPACTION = "IN_MEMORY_COMPACTION";
063  private static final Bytes IN_MEMORY_COMPACTION_BYTES =
064    new Bytes(Bytes.toBytes(IN_MEMORY_COMPACTION));
065
066  @InterfaceAudience.Private
067  public static final String IN_MEMORY = HConstants.IN_MEMORY;
068  private static final Bytes IN_MEMORY_BYTES = new Bytes(Bytes.toBytes(IN_MEMORY));
069
070  // These constants are used as FileInfo keys
071  @InterfaceAudience.Private
072  public static final String COMPRESSION = "COMPRESSION";
073  private static final Bytes COMPRESSION_BYTES = new Bytes(Bytes.toBytes(COMPRESSION));
074  @InterfaceAudience.Private
075  public static final String COMPRESSION_COMPACT = "COMPRESSION_COMPACT";
076  private static final Bytes COMPRESSION_COMPACT_BYTES =
077    new Bytes(Bytes.toBytes(COMPRESSION_COMPACT));
078  public static final String COMPRESSION_COMPACT_MAJOR = "COMPRESSION_COMPACT_MAJOR";
079  private static final Bytes COMPRESSION_COMPACT_MAJOR_BYTES =
080    new Bytes(Bytes.toBytes(COMPRESSION_COMPACT_MAJOR));
081  public static final String COMPRESSION_COMPACT_MINOR = "COMPRESSION_COMPACT_MINOR";
082  private static final Bytes COMPRESSION_COMPACT_MINOR_BYTES =
083    new Bytes(Bytes.toBytes(COMPRESSION_COMPACT_MINOR));
084  @InterfaceAudience.Private
085  public static final String DATA_BLOCK_ENCODING = "DATA_BLOCK_ENCODING";
086  private static final Bytes DATA_BLOCK_ENCODING_BYTES =
087    new Bytes(Bytes.toBytes(DATA_BLOCK_ENCODING));
088  @InterfaceAudience.Private
089  public static final String INDEX_BLOCK_ENCODING = "INDEX_BLOCK_ENCODING";
090  private static final Bytes INDEX_BLOCK_ENCODING_BYTES =
091    new Bytes(Bytes.toBytes(INDEX_BLOCK_ENCODING));
092  /**
093   * Key for the BLOCKCACHE attribute. A more exact name would be CACHE_DATA_ON_READ because this
094   * flag sets whether or not we cache DATA blocks. We always cache INDEX and BLOOM blocks; caching
095   * these blocks cannot be disabled.
096   */
097  @InterfaceAudience.Private
098  public static final String BLOCKCACHE = "BLOCKCACHE";
099  private static final Bytes BLOCKCACHE_BYTES = new Bytes(Bytes.toBytes(BLOCKCACHE));
100  @InterfaceAudience.Private
101  public static final String CACHE_DATA_ON_WRITE = "CACHE_DATA_ON_WRITE";
102  private static final Bytes CACHE_DATA_ON_WRITE_BYTES =
103    new Bytes(Bytes.toBytes(CACHE_DATA_ON_WRITE));
104  @InterfaceAudience.Private
105  public static final String CACHE_INDEX_ON_WRITE = "CACHE_INDEX_ON_WRITE";
106  private static final Bytes CACHE_INDEX_ON_WRITE_BYTES =
107    new Bytes(Bytes.toBytes(CACHE_INDEX_ON_WRITE));
108  @InterfaceAudience.Private
109  public static final String CACHE_BLOOMS_ON_WRITE = "CACHE_BLOOMS_ON_WRITE";
110  private static final Bytes CACHE_BLOOMS_ON_WRITE_BYTES =
111    new Bytes(Bytes.toBytes(CACHE_BLOOMS_ON_WRITE));
112  @InterfaceAudience.Private
113  public static final String EVICT_BLOCKS_ON_CLOSE = "EVICT_BLOCKS_ON_CLOSE";
114  private static final Bytes EVICT_BLOCKS_ON_CLOSE_BYTES =
115    new Bytes(Bytes.toBytes(EVICT_BLOCKS_ON_CLOSE));
116
117  /**
118   * Key for the PREFETCH_BLOCKS_ON_OPEN attribute. If set, all INDEX, BLOOM, and DATA blocks of
119   * HFiles belonging to this family will be loaded into the cache as soon as the file is opened.
120   * These loads will not count as cache misses.
121   */
122  @InterfaceAudience.Private
123  public static final String PREFETCH_BLOCKS_ON_OPEN = "PREFETCH_BLOCKS_ON_OPEN";
124  private static final Bytes PREFETCH_BLOCKS_ON_OPEN_BYTES =
125    new Bytes(Bytes.toBytes(PREFETCH_BLOCKS_ON_OPEN));
126
127  /**
128   * Size of storefile/hfile 'blocks'. Default is {@link #DEFAULT_BLOCKSIZE}. Use smaller block
129   * sizes for faster random-access at expense of larger indices (more memory consumption). Note
130   * that this is a soft limit and that blocks have overhead (metadata, CRCs) so blocks will tend to
131   * be the size specified here and then some; i.e. don't expect that setting BLOCKSIZE=4k means
132   * hbase data will align with an SSDs 4k page accesses (TODO).
133   */
134  @InterfaceAudience.Private
135  public static final String BLOCKSIZE = "BLOCKSIZE";
136  private static final Bytes BLOCKSIZE_BYTES = new Bytes(Bytes.toBytes(BLOCKSIZE));
137
138  @InterfaceAudience.Private
139  public static final String TTL = "TTL";
140  private static final Bytes TTL_BYTES = new Bytes(Bytes.toBytes(TTL));
141  @InterfaceAudience.Private
142  public static final String BLOOMFILTER = "BLOOMFILTER";
143  private static final Bytes BLOOMFILTER_BYTES = new Bytes(Bytes.toBytes(BLOOMFILTER));
144  @InterfaceAudience.Private
145  public static final String REPLICATION_SCOPE = "REPLICATION_SCOPE";
146  @InterfaceAudience.Private
147  public static final String MAX_VERSIONS = HConstants.VERSIONS;
148  private static final Bytes MAX_VERSIONS_BYTES = new Bytes(Bytes.toBytes(MAX_VERSIONS));
149  @InterfaceAudience.Private
150  public static final String MIN_VERSIONS = "MIN_VERSIONS";
151  private static final Bytes MIN_VERSIONS_BYTES = new Bytes(Bytes.toBytes(MIN_VERSIONS));
152  /**
153   * Retain all cells across flushes and compactions even if they fall behind a delete tombstone. To
154   * see all retained cells, do a 'raw' scan; see Scan#setRaw or pass RAW => true attribute in
155   * the shell.
156   */
157  @InterfaceAudience.Private
158  public static final String KEEP_DELETED_CELLS = "KEEP_DELETED_CELLS";
159  private static final Bytes KEEP_DELETED_CELLS_BYTES =
160    new Bytes(Bytes.toBytes(KEEP_DELETED_CELLS));
161  @InterfaceAudience.Private
162  public static final String COMPRESS_TAGS = "COMPRESS_TAGS";
163  private static final Bytes COMPRESS_TAGS_BYTES = new Bytes(Bytes.toBytes(COMPRESS_TAGS));
164  @InterfaceAudience.Private
165  public static final String ENCRYPTION = "ENCRYPTION";
166  private static final Bytes ENCRYPTION_BYTES = new Bytes(Bytes.toBytes(ENCRYPTION));
167  @InterfaceAudience.Private
168  public static final String ENCRYPTION_KEY = "ENCRYPTION_KEY";
169  private static final Bytes ENCRYPTION_KEY_BYTES = new Bytes(Bytes.toBytes(ENCRYPTION_KEY));
170
171  private static final boolean DEFAULT_MOB = false;
172  @InterfaceAudience.Private
173  public static final String IS_MOB = "IS_MOB";
174  private static final Bytes IS_MOB_BYTES = new Bytes(Bytes.toBytes(IS_MOB));
175  @InterfaceAudience.Private
176  public static final String MOB_THRESHOLD = "MOB_THRESHOLD";
177  private static final Bytes MOB_THRESHOLD_BYTES = new Bytes(Bytes.toBytes(MOB_THRESHOLD));
178  public static final long DEFAULT_MOB_THRESHOLD = 100 * 1024; // 100k
179  @InterfaceAudience.Private
180  public static final String MOB_COMPACT_PARTITION_POLICY = "MOB_COMPACT_PARTITION_POLICY";
181  private static final Bytes MOB_COMPACT_PARTITION_POLICY_BYTES =
182    new Bytes(Bytes.toBytes(MOB_COMPACT_PARTITION_POLICY));
183  public static final MobCompactPartitionPolicy DEFAULT_MOB_COMPACT_PARTITION_POLICY =
184    MobCompactPartitionPolicy.DAILY;
185  @InterfaceAudience.Private
186  public static final String DFS_REPLICATION = "DFS_REPLICATION";
187  private static final Bytes DFS_REPLICATION_BYTES = new Bytes(Bytes.toBytes(DFS_REPLICATION));
188  public static final short DEFAULT_DFS_REPLICATION = 0;
189  @InterfaceAudience.Private
190  public static final String STORAGE_POLICY = "STORAGE_POLICY";
191  private static final Bytes STORAGE_POLICY_BYTES = new Bytes(Bytes.toBytes(STORAGE_POLICY));
192
193  public static final String NEW_VERSION_BEHAVIOR = "NEW_VERSION_BEHAVIOR";
194  private static final Bytes NEW_VERSION_BEHAVIOR_BYTES =
195    new Bytes(Bytes.toBytes(NEW_VERSION_BEHAVIOR));
196  public static final boolean DEFAULT_NEW_VERSION_BEHAVIOR = false;
197  /**
198   * Default compression type.
199   */
200  public static final Compression.Algorithm DEFAULT_COMPRESSION = Compression.Algorithm.NONE;
201
202  /**
203   * Default data block encoding algorithm.
204   */
205  public static final DataBlockEncoding DEFAULT_DATA_BLOCK_ENCODING = DataBlockEncoding.NONE;
206
207  /**
208   * Default index block encoding algorithm.
209   */
210  public static final IndexBlockEncoding DEFAULT_INDEX_BLOCK_ENCODING = IndexBlockEncoding.NONE;
211
212  /**
213   * Default number of versions of a record to keep.
214   */
215  public static final int DEFAULT_MAX_VERSIONS = 1;
216
217  /**
218   * Default is not to keep a minimum of versions.
219   */
220  public static final int DEFAULT_MIN_VERSIONS = 0;
221
222  /**
223   * Default setting for whether to try and serve this column family from memory or not.
224   */
225  public static final boolean DEFAULT_IN_MEMORY = false;
226
227  /**
228   * Default setting for preventing deleted from being collected immediately.
229   */
230  public static final KeepDeletedCells DEFAULT_KEEP_DELETED = KeepDeletedCells.FALSE;
231
232  /**
233   * Default setting for whether to use a block cache or not.
234   */
235  public static final boolean DEFAULT_BLOCKCACHE = true;
236
237  /**
238   * Default setting for whether to cache data blocks on write if block caching is enabled.
239   */
240  public static final boolean DEFAULT_CACHE_DATA_ON_WRITE = false;
241
242  /**
243   * Default setting for whether to cache index blocks on write if block caching is enabled.
244   */
245  public static final boolean DEFAULT_CACHE_INDEX_ON_WRITE = false;
246
247  /**
248   * Default size of blocks in files stored to the filesytem (hfiles).
249   */
250  public static final int DEFAULT_BLOCKSIZE = HConstants.DEFAULT_BLOCKSIZE;
251
252  /**
253   * Default setting for whether or not to use bloomfilters.
254   */
255  public static final BloomType DEFAULT_BLOOMFILTER = BloomType.ROW;
256
257  /**
258   * Default setting for whether to cache bloom filter blocks on write if block caching is enabled.
259   */
260  public static final boolean DEFAULT_CACHE_BLOOMS_ON_WRITE = false;
261
262  /**
263   * Default time to live of cell contents.
264   */
265  public static final int DEFAULT_TTL = HConstants.FOREVER;
266
267  /**
268   * Default scope.
269   */
270  public static final int DEFAULT_REPLICATION_SCOPE = HConstants.REPLICATION_SCOPE_LOCAL;
271
272  /**
273   * Default setting for whether to evict cached blocks from the blockcache on close.
274   */
275  public static final boolean DEFAULT_EVICT_BLOCKS_ON_CLOSE = false;
276
277  /**
278   * Default compress tags along with any type of DataBlockEncoding.
279   */
280  public static final boolean DEFAULT_COMPRESS_TAGS = true;
281
282  /*
283   * Default setting for whether to prefetch blocks into the blockcache on open.
284   */
285  public static final boolean DEFAULT_PREFETCH_BLOCKS_ON_OPEN = false;
286
287  private final static Map<String, String> DEFAULT_VALUES = new HashMap<>();
288
289  private static Map<Bytes, Bytes> getDefaultValuesBytes() {
290    Map<Bytes, Bytes> values = new HashMap<>();
291    DEFAULT_VALUES
292      .forEach((k, v) -> values.put(new Bytes(Bytes.toBytes(k)), new Bytes(Bytes.toBytes(v))));
293    return values;
294  }
295
296  public static Map<String, String> getDefaultValues() {
297    return Collections.unmodifiableMap(DEFAULT_VALUES);
298  }
299
300  private final static Set<Bytes> RESERVED_KEYWORDS = new HashSet<>();
301
302  static {
303    DEFAULT_VALUES.put(BLOOMFILTER, DEFAULT_BLOOMFILTER.name());
304    DEFAULT_VALUES.put(REPLICATION_SCOPE, String.valueOf(DEFAULT_REPLICATION_SCOPE));
305    DEFAULT_VALUES.put(MAX_VERSIONS, String.valueOf(DEFAULT_MAX_VERSIONS));
306    DEFAULT_VALUES.put(MIN_VERSIONS, String.valueOf(DEFAULT_MIN_VERSIONS));
307    DEFAULT_VALUES.put(COMPRESSION, DEFAULT_COMPRESSION.name());
308    DEFAULT_VALUES.put(TTL, String.valueOf(DEFAULT_TTL));
309    DEFAULT_VALUES.put(BLOCKSIZE, String.valueOf(DEFAULT_BLOCKSIZE));
310    DEFAULT_VALUES.put(IN_MEMORY, String.valueOf(DEFAULT_IN_MEMORY));
311    DEFAULT_VALUES.put(BLOCKCACHE, String.valueOf(DEFAULT_BLOCKCACHE));
312    DEFAULT_VALUES.put(KEEP_DELETED_CELLS, String.valueOf(DEFAULT_KEEP_DELETED));
313    DEFAULT_VALUES.put(DATA_BLOCK_ENCODING, String.valueOf(DEFAULT_DATA_BLOCK_ENCODING));
314    DEFAULT_VALUES.put(INDEX_BLOCK_ENCODING, String.valueOf(DEFAULT_INDEX_BLOCK_ENCODING));
315    // Do NOT add this key/value by default. NEW_VERSION_BEHAVIOR is NOT defined in hbase1 so
316    // it is not possible to make an hbase1 HCD the same as an hbase2 HCD and so the replication
317    // compare of schemas will fail. It is OK not adding the below to the initial map because of
318    // fetch of this value, we will check for null and if null will return the default.
319    // DEFAULT_VALUES.put(NEW_VERSION_BEHAVIOR, String.valueOf(DEFAULT_NEW_VERSION_BEHAVIOR));
320    DEFAULT_VALUES.keySet().forEach(s -> RESERVED_KEYWORDS.add(new Bytes(Bytes.toBytes(s))));
321    RESERVED_KEYWORDS.add(new Bytes(Bytes.toBytes(ENCRYPTION)));
322    RESERVED_KEYWORDS.add(new Bytes(Bytes.toBytes(ENCRYPTION_KEY)));
323    RESERVED_KEYWORDS.add(new Bytes(Bytes.toBytes(IS_MOB)));
324    RESERVED_KEYWORDS.add(new Bytes(Bytes.toBytes(MOB_THRESHOLD)));
325    RESERVED_KEYWORDS.add(new Bytes(Bytes.toBytes(MOB_COMPACT_PARTITION_POLICY)));
326  }
327
328  public static Unit getUnit(String key) {
329    /* TTL for now, we can add more as we need */
330    switch (key) {
331      case TTL:
332        return Unit.TIME_INTERVAL;
333      case BLOCKSIZE:
334        return Unit.BYTE;
335      default:
336        return Unit.NONE;
337    }
338  }
339
340  /**
341   * Check if the column family name is legal.
342   * @param b Family name.
343   * @return <code>b</code>
344   * @throws IllegalArgumentException If not null and not a legitimate family name: i.e. 'printable'
345   *                                  and ends in a ':' (Null passes are allowed because
346   *                                  <code>b</code> can be null when deserializing). Cannot start
347   *                                  with a '.' either. Also Family can not be an empty value or
348   *                                  equal "recovered.edits".
349   */
350  public static byte[] isLegalColumnFamilyName(final byte[] b) {
351    if (b == null) {
352      return null;
353    }
354    Preconditions.checkArgument(b.length != 0, "Column Family name can not be empty");
355    if (b[0] == '.') {
356      throw new IllegalArgumentException(
357        "Column Family names cannot start with a " + "period: " + Bytes.toString(b));
358    }
359    for (int i = 0; i < b.length; i++) {
360      if (Character.isISOControl(b[i]) || b[i] == ':' || b[i] == '\\' || b[i] == '/') {
361        throw new IllegalArgumentException("Illegal character <" + b[i]
362          + ">. Column Family names cannot contain control characters or colons: "
363          + Bytes.toString(b));
364      }
365    }
366    byte[] recoveredEdit = Bytes.toBytes(HConstants.RECOVERED_EDITS_DIR);
367    if (Bytes.equals(recoveredEdit, b)) {
368      throw new IllegalArgumentException(
369        "Column Family name cannot be: " + HConstants.RECOVERED_EDITS_DIR);
370    }
371    return b;
372  }
373
374  private final ModifyableColumnFamilyDescriptor desc;
375
376  public static ColumnFamilyDescriptor parseFrom(final byte[] pbBytes)
377    throws DeserializationException {
378    return ModifyableColumnFamilyDescriptor.parseFrom(pbBytes);
379  }
380
381  public static ColumnFamilyDescriptorBuilder newBuilder(final byte[] name) {
382    return new ColumnFamilyDescriptorBuilder(name);
383  }
384
385  public static ColumnFamilyDescriptorBuilder newBuilder(final ColumnFamilyDescriptor desc) {
386    return new ColumnFamilyDescriptorBuilder(desc);
387  }
388
389  public static ColumnFamilyDescriptor copy(ColumnFamilyDescriptor desc) {
390    return new ModifyableColumnFamilyDescriptor(desc);
391  }
392
393  public static ColumnFamilyDescriptor of(String name) {
394    return of(Bytes.toBytes(name));
395  }
396
397  public static ColumnFamilyDescriptor of(byte[] name) {
398    return newBuilder(name).build();
399  }
400
401  private ColumnFamilyDescriptorBuilder(final byte[] name) {
402    this.desc = new ModifyableColumnFamilyDescriptor(name);
403  }
404
405  private ColumnFamilyDescriptorBuilder(final ColumnFamilyDescriptor desc) {
406    this.desc = new ModifyableColumnFamilyDescriptor(desc);
407  }
408
409  /**
410   * Serialize the table descriptor to a byte array.
411   * @param desc The table descriptor to serialize
412   * @return This instance serialized with pb with pb magic prefix
413   */
414  public static byte[] toByteArray(ColumnFamilyDescriptor desc) {
415    if (desc instanceof ModifyableColumnFamilyDescriptor) {
416      return ((ModifyableColumnFamilyDescriptor) desc).toByteArray();
417    }
418    return new ModifyableColumnFamilyDescriptor(desc).toByteArray();
419  }
420
421  public ColumnFamilyDescriptor build() {
422    return new ModifyableColumnFamilyDescriptor(desc);
423  }
424
425  public ColumnFamilyDescriptorBuilder removeConfiguration(String key) {
426    desc.removeConfiguration(key);
427    return this;
428  }
429
430  public String getNameAsString() {
431    return desc.getNameAsString();
432  }
433
434  public ColumnFamilyDescriptorBuilder setBlockCacheEnabled(boolean value) {
435    desc.setBlockCacheEnabled(value);
436    return this;
437  }
438
439  public ColumnFamilyDescriptorBuilder setBlocksize(int value) {
440    desc.setBlocksize(value);
441    return this;
442  }
443
444  public ColumnFamilyDescriptorBuilder setBlocksize(String value) throws HBaseException {
445    desc.setBlocksize(value);
446    return this;
447  }
448
449  public ColumnFamilyDescriptorBuilder setBloomFilterType(final BloomType value) {
450    desc.setBloomFilterType(value);
451    return this;
452  }
453
454  public ColumnFamilyDescriptorBuilder setCacheBloomsOnWrite(boolean value) {
455    desc.setCacheBloomsOnWrite(value);
456    return this;
457  }
458
459  public ColumnFamilyDescriptorBuilder setCacheDataOnWrite(boolean value) {
460    desc.setCacheDataOnWrite(value);
461    return this;
462  }
463
464  public ColumnFamilyDescriptorBuilder setCacheIndexesOnWrite(final boolean value) {
465    desc.setCacheIndexesOnWrite(value);
466    return this;
467  }
468
469  public ColumnFamilyDescriptorBuilder setCompactionCompressionType(Compression.Algorithm value) {
470    desc.setCompactionCompressionType(value);
471    return this;
472  }
473
474  public ColumnFamilyDescriptorBuilder
475    setMajorCompactionCompressionType(Compression.Algorithm value) {
476    desc.setMajorCompactionCompressionType(value);
477    return this;
478  }
479
480  public ColumnFamilyDescriptorBuilder
481    setMinorCompactionCompressionType(Compression.Algorithm value) {
482    desc.setMinorCompactionCompressionType(value);
483    return this;
484  }
485
486  public ColumnFamilyDescriptorBuilder setCompressTags(boolean value) {
487    desc.setCompressTags(value);
488    return this;
489  }
490
491  public ColumnFamilyDescriptorBuilder setCompressionType(Compression.Algorithm value) {
492    desc.setCompressionType(value);
493    return this;
494  }
495
496  public Compression.Algorithm getCompressionType() {
497    return desc.getCompressionType();
498  }
499
500  public ColumnFamilyDescriptorBuilder setConfiguration(final String key, final String value) {
501    desc.setConfiguration(key, value);
502    return this;
503  }
504
505  public ColumnFamilyDescriptorBuilder setDFSReplication(short value) {
506    desc.setDFSReplication(value);
507    return this;
508  }
509
510  public ColumnFamilyDescriptorBuilder setDataBlockEncoding(DataBlockEncoding value) {
511    desc.setDataBlockEncoding(value);
512    return this;
513  }
514
515  public ColumnFamilyDescriptorBuilder setIndexBlockEncoding(IndexBlockEncoding value) {
516    desc.setIndexBlockEncoding(value);
517    return this;
518  }
519
520  public ColumnFamilyDescriptorBuilder setEncryptionKey(final byte[] value) {
521    desc.setEncryptionKey(value);
522    return this;
523  }
524
525  public ColumnFamilyDescriptorBuilder setEncryptionType(String value) {
526    desc.setEncryptionType(value);
527    return this;
528  }
529
530  public ColumnFamilyDescriptorBuilder setEvictBlocksOnClose(boolean value) {
531    desc.setEvictBlocksOnClose(value);
532    return this;
533  }
534
535  public ColumnFamilyDescriptorBuilder setInMemory(final boolean value) {
536    desc.setInMemory(value);
537    return this;
538  }
539
540  public ColumnFamilyDescriptorBuilder setInMemoryCompaction(final MemoryCompactionPolicy value) {
541    desc.setInMemoryCompaction(value);
542    return this;
543  }
544
545  public ColumnFamilyDescriptorBuilder setKeepDeletedCells(KeepDeletedCells value) {
546    desc.setKeepDeletedCells(value);
547    return this;
548  }
549
550  public ColumnFamilyDescriptorBuilder setMaxVersions(final int value) {
551    desc.setMaxVersions(value);
552    return this;
553  }
554
555  public ColumnFamilyDescriptorBuilder setMinVersions(final int value) {
556    desc.setMinVersions(value);
557    return this;
558  }
559
560  public ColumnFamilyDescriptorBuilder
561    setMobCompactPartitionPolicy(final MobCompactPartitionPolicy value) {
562    desc.setMobCompactPartitionPolicy(value);
563    return this;
564  }
565
566  public ColumnFamilyDescriptorBuilder setMobEnabled(final boolean value) {
567    desc.setMobEnabled(value);
568    return this;
569  }
570
571  public ColumnFamilyDescriptorBuilder setMobThreshold(final long value) {
572    desc.setMobThreshold(value);
573    return this;
574  }
575
576  public ColumnFamilyDescriptorBuilder setPrefetchBlocksOnOpen(final boolean value) {
577    desc.setPrefetchBlocksOnOpen(value);
578    return this;
579  }
580
581  public ColumnFamilyDescriptorBuilder setScope(final int value) {
582    desc.setScope(value);
583    return this;
584  }
585
586  public ColumnFamilyDescriptorBuilder setStoragePolicy(final String value) {
587    desc.setStoragePolicy(value);
588    return this;
589  }
590
591  public ColumnFamilyDescriptorBuilder setTimeToLive(final int value) {
592    desc.setTimeToLive(value);
593    return this;
594  }
595
596  public ColumnFamilyDescriptorBuilder setTimeToLive(final String value) throws HBaseException {
597    desc.setTimeToLive(value);
598    return this;
599  }
600
601  public ColumnFamilyDescriptorBuilder setNewVersionBehavior(final boolean value) {
602    desc.setNewVersionBehavior(value);
603    return this;
604  }
605
606  public ColumnFamilyDescriptorBuilder setValue(final Bytes key, final Bytes value) {
607    desc.setValue(key, value);
608    return this;
609  }
610
611  public ColumnFamilyDescriptorBuilder setValue(final byte[] key, final byte[] value) {
612    desc.setValue(key, value);
613    return this;
614  }
615
616  public ColumnFamilyDescriptorBuilder setValue(final String key, final String value) {
617    desc.setValue(key, value);
618    return this;
619  }
620
621  public ColumnFamilyDescriptorBuilder setVersionsWithTimeToLive(final int retentionInterval,
622    final int versionAfterInterval) {
623    desc.setVersionsWithTimeToLive(retentionInterval, versionAfterInterval);
624    return this;
625  }
626
627  /**
628   * An ModifyableFamilyDescriptor contains information about a column family such as the number of
629   * versions, compression settings, etc. It is used as input when creating a table or adding a
630   * column. TODO: make this package-private after removing the HColumnDescriptor
631   */
632  @InterfaceAudience.Private
633  public static class ModifyableColumnFamilyDescriptor
634    implements ColumnFamilyDescriptor, Comparable<ModifyableColumnFamilyDescriptor> {
635
636    // Column family name
637    private final byte[] name;
638
639    // Column metadata
640    private final Map<Bytes, Bytes> values = new HashMap<>();
641
642    /**
643     * A map which holds the configuration specific to the column family. The keys of the map have
644     * the same names as config keys and override the defaults with cf-specific settings. Example
645     * usage may be for compactions, etc.
646     */
647    private final Map<String, String> configuration = new HashMap<>();
648
649    /**
650     * Construct a column descriptor specifying only the family name The other attributes are
651     * defaulted.
652     * @param name Column family name. Must be 'printable' -- digit or letter -- and may not contain
653     *             a <code>:</code> TODO: make this private after the HCD is removed.
654     */
655    @InterfaceAudience.Private
656    public ModifyableColumnFamilyDescriptor(final byte[] name) {
657      this(isLegalColumnFamilyName(name), getDefaultValuesBytes(), Collections.emptyMap());
658    }
659
660    /**
661     * Constructor. Makes a deep copy of the supplied descriptor. TODO: make this private after the
662     * HCD is removed.
663     * @param desc The descriptor.
664     */
665    @InterfaceAudience.Private
666    public ModifyableColumnFamilyDescriptor(ColumnFamilyDescriptor desc) {
667      this(desc.getName(), desc.getValues(), desc.getConfiguration());
668    }
669
670    private ModifyableColumnFamilyDescriptor(byte[] name, Map<Bytes, Bytes> values,
671      Map<String, String> config) {
672      this.name = name;
673      this.values.putAll(values);
674      this.configuration.putAll(config);
675    }
676
677    @Override
678    public byte[] getName() {
679      return Bytes.copy(name);
680    }
681
682    @Override
683    public String getNameAsString() {
684      return Bytes.toString(name);
685    }
686
687    @Override
688    public Bytes getValue(Bytes key) {
689      return values.get(key);
690    }
691
692    @Override
693    public byte[] getValue(byte[] key) {
694      Bytes value = values.get(new Bytes(key));
695      return value == null ? null : value.get();
696    }
697
698    @Override
699    public String getValue(String key) {
700      Bytes rval = values.get(new Bytes(Bytes.toBytes(key)));
701      return rval == null ? null : Bytes.toString(rval.get(), rval.getOffset(), rval.getLength());
702    }
703
704    @Override
705    public Map<Bytes, Bytes> getValues() {
706      return Collections.unmodifiableMap(values);
707    }
708
709    public ModifyableColumnFamilyDescriptor setValue(byte[] key, byte[] value) {
710      return setValue(toBytesOrNull(key, Function.identity()),
711        toBytesOrNull(value, Function.identity()));
712    }
713
714    public ModifyableColumnFamilyDescriptor setValue(String key, String value) {
715      return setValue(toBytesOrNull(key, Bytes::toBytes), toBytesOrNull(value, Bytes::toBytes));
716    }
717
718    private ModifyableColumnFamilyDescriptor setValue(Bytes key, String value) {
719      return setValue(key, toBytesOrNull(value, Bytes::toBytes));
720    }
721
722    private ModifyableColumnFamilyDescriptor setValue(Bytes key, Bytes value) {
723      if (value == null || value.getLength() == 0) {
724        values.remove(key);
725      } else {
726        values.put(key, value);
727      }
728      return this;
729    }
730
731    public ModifyableColumnFamilyDescriptor removeValue(final Bytes key) {
732      return setValue(key, (Bytes) null);
733    }
734
735    private static <T> Bytes toBytesOrNull(T t, Function<T, byte[]> f) {
736      if (t == null) {
737        return null;
738      } else {
739        return new Bytes(f.apply(t));
740      }
741    }
742
743    private <T> T getStringOrDefault(Bytes key, Function<String, T> function, T defaultValue) {
744      return getOrDefault(key, b -> function.apply(Bytes.toString(b)), defaultValue);
745    }
746
747    private <T> T getOrDefault(Bytes key, Function<byte[], T> function, T defaultValue) {
748      Bytes value = values.get(key);
749      if (value == null) {
750        return defaultValue;
751      } else {
752        return function.apply(value.get());
753      }
754    }
755
756    @Override
757    public int getMaxVersions() {
758      return getStringOrDefault(MAX_VERSIONS_BYTES, Integer::parseInt, DEFAULT_MAX_VERSIONS);
759    }
760
761    /**
762     * Set the maximum number of versions to retain.
763     * @param maxVersions maximum number of versions
764     * @return this (for chained invocation)
765     */
766    public ModifyableColumnFamilyDescriptor setMaxVersions(int maxVersions) {
767      if (maxVersions <= 0) {
768        // TODO: Allow maxVersion of 0 to be the way you say "Keep all versions".
769        // Until there is support, consider 0 or < 0 -- a configuration error.
770        throw new IllegalArgumentException("Maximum versions must be positive");
771      }
772      if (maxVersions < this.getMinVersions()) {
773        throw new IllegalArgumentException(
774          "Set MaxVersion to " + maxVersions + " while minVersion is " + this.getMinVersions()
775            + ". Maximum versions must be >= minimum versions ");
776      }
777      setValue(MAX_VERSIONS_BYTES, Integer.toString(maxVersions));
778      return this;
779    }
780
781    /**
782     * Set minimum and maximum versions to keep.
783     * @param minVersions minimal number of versions
784     * @param maxVersions maximum number of versions
785     * @return this (for chained invocation)
786     */
787    public ModifyableColumnFamilyDescriptor setVersions(int minVersions, int maxVersions) {
788      if (minVersions <= 0) {
789        // TODO: Allow minVersion and maxVersion of 0 to be the way you say "Keep all versions".
790        // Until there is support, consider 0 or < 0 -- a configuration error.
791        throw new IllegalArgumentException("Minimum versions must be positive");
792      }
793
794      if (maxVersions < minVersions) {
795        throw new IllegalArgumentException(
796          "Unable to set MaxVersion to " + maxVersions + " and set MinVersion to " + minVersions
797            + ", as maximum versions must be >= minimum versions.");
798      }
799      setMinVersions(minVersions);
800      setMaxVersions(maxVersions);
801      return this;
802    }
803
804    @Override
805    public int getBlocksize() {
806      return getStringOrDefault(BLOCKSIZE_BYTES, Integer::valueOf, DEFAULT_BLOCKSIZE);
807    }
808
809    public ModifyableColumnFamilyDescriptor setBlocksize(int s) {
810      return setValue(BLOCKSIZE_BYTES, Integer.toString(s));
811    }
812
813    public ModifyableColumnFamilyDescriptor setBlocksize(String blocksize) throws HBaseException {
814      return setBlocksize(
815        Integer.parseInt(PrettyPrinter.valueOf(blocksize, PrettyPrinter.Unit.BYTE)));
816    }
817
818    @Override
819    public Compression.Algorithm getCompressionType() {
820      return getStringOrDefault(COMPRESSION_BYTES,
821        n -> Compression.Algorithm.valueOf(n.toUpperCase()), DEFAULT_COMPRESSION);
822    }
823
824    /**
825     * Compression types supported in hbase. LZO is not bundled as part of the hbase distribution.
826     * See <a href="http://wiki.apache.org/hadoop/UsingLzoCompression">LZO Compression</a> for how
827     * to enable it.
828     * @param type Compression type setting.
829     * @return this (for chained invocation)
830     */
831    public ModifyableColumnFamilyDescriptor setCompressionType(Compression.Algorithm type) {
832      return setValue(COMPRESSION_BYTES, type.name());
833    }
834
835    @Override
836    public DataBlockEncoding getDataBlockEncoding() {
837      return getStringOrDefault(DATA_BLOCK_ENCODING_BYTES,
838        n -> DataBlockEncoding.valueOf(n.toUpperCase()), DataBlockEncoding.NONE);
839    }
840
841    /**
842     * Set data block encoding algorithm used in block cache.
843     * @param type What kind of data block encoding will be used.
844     * @return this (for chained invocation)
845     */
846    public ModifyableColumnFamilyDescriptor setDataBlockEncoding(DataBlockEncoding type) {
847      return setValue(DATA_BLOCK_ENCODING_BYTES,
848        type == null ? DataBlockEncoding.NONE.name() : type.name());
849    }
850
851    @Override
852    public IndexBlockEncoding getIndexBlockEncoding() {
853      return getStringOrDefault(INDEX_BLOCK_ENCODING_BYTES,
854        n -> IndexBlockEncoding.valueOf(n.toUpperCase()), IndexBlockEncoding.NONE);
855    }
856
857    /**
858     * Set index block encoding algorithm used in block cache.
859     * @param type What kind of index block encoding will be used.
860     * @return this (for chained invocation)
861     */
862    public ModifyableColumnFamilyDescriptor setIndexBlockEncoding(IndexBlockEncoding type) {
863      return setValue(INDEX_BLOCK_ENCODING_BYTES,
864        type == null ? IndexBlockEncoding.NONE.name() : type.name());
865    }
866
867    /**
868     * Set whether the tags should be compressed along with DataBlockEncoding. When no
869     * DataBlockEncoding is been used, this is having no effect.
870     * @return this (for chained invocation)
871     */
872    public ModifyableColumnFamilyDescriptor setCompressTags(boolean compressTags) {
873      return setValue(COMPRESS_TAGS_BYTES, String.valueOf(compressTags));
874    }
875
876    @Override
877    public boolean isCompressTags() {
878      return getStringOrDefault(COMPRESS_TAGS_BYTES, Boolean::valueOf, DEFAULT_COMPRESS_TAGS);
879    }
880
881    @Override
882    public Compression.Algorithm getCompactionCompressionType() {
883      return getStringOrDefault(COMPRESSION_COMPACT_BYTES,
884        n -> Compression.Algorithm.valueOf(n.toUpperCase()), getCompressionType());
885    }
886
887    @Override
888    public Compression.Algorithm getMajorCompactionCompressionType() {
889      return getStringOrDefault(COMPRESSION_COMPACT_MAJOR_BYTES,
890        n -> Compression.Algorithm.valueOf(n.toUpperCase()), getCompactionCompressionType());
891    }
892
893    @Override
894    public Compression.Algorithm getMinorCompactionCompressionType() {
895      return getStringOrDefault(COMPRESSION_COMPACT_MINOR_BYTES,
896        n -> Compression.Algorithm.valueOf(n.toUpperCase()), getCompactionCompressionType());
897    }
898
899    /**
900     * Compression types supported in hbase. LZO is not bundled as part of the hbase distribution.
901     * See <a href="http://wiki.apache.org/hadoop/UsingLzoCompression">LZO Compression</a> for how
902     * to enable it.
903     * @param type Compression type setting.
904     * @return this (for chained invocation)
905     */
906    public ModifyableColumnFamilyDescriptor
907      setCompactionCompressionType(Compression.Algorithm type) {
908      return setValue(COMPRESSION_COMPACT_BYTES, type.name());
909    }
910
911    public ModifyableColumnFamilyDescriptor
912      setMajorCompactionCompressionType(Compression.Algorithm type) {
913      return setValue(COMPRESSION_COMPACT_MAJOR_BYTES, type.name());
914    }
915
916    public ModifyableColumnFamilyDescriptor
917      setMinorCompactionCompressionType(Compression.Algorithm type) {
918      return setValue(COMPRESSION_COMPACT_MINOR_BYTES, type.name());
919    }
920
921    @Override
922    public boolean isInMemory() {
923      return getStringOrDefault(IN_MEMORY_BYTES, Boolean::valueOf, DEFAULT_IN_MEMORY);
924    }
925
926    /**
927     * Set the inMemory flag
928     * @param inMemory True if we are to favor keeping all values for this column family in the
929     *                 HRegionServer cache
930     * @return this (for chained invocation)
931     */
932    public ModifyableColumnFamilyDescriptor setInMemory(boolean inMemory) {
933      return setValue(IN_MEMORY_BYTES, Boolean.toString(inMemory));
934    }
935
936    @Override
937    public MemoryCompactionPolicy getInMemoryCompaction() {
938      return getStringOrDefault(IN_MEMORY_COMPACTION_BYTES,
939        n -> MemoryCompactionPolicy.valueOf(n.toUpperCase()), null);
940    }
941
942    public ModifyableColumnFamilyDescriptor
943      setInMemoryCompaction(MemoryCompactionPolicy inMemoryCompaction) {
944      return setValue(IN_MEMORY_COMPACTION_BYTES, inMemoryCompaction.name());
945    }
946
947    @Override
948    public KeepDeletedCells getKeepDeletedCells() {
949      return getStringOrDefault(KEEP_DELETED_CELLS_BYTES, KeepDeletedCells::getValue,
950        DEFAULT_KEEP_DELETED);
951    }
952
953    public ModifyableColumnFamilyDescriptor setKeepDeletedCells(KeepDeletedCells keepDeletedCells) {
954      return setValue(KEEP_DELETED_CELLS_BYTES, keepDeletedCells.name());
955    }
956
957    /**
958     * By default, HBase only consider timestamp in versions. So a previous Delete with higher ts
959     * will mask a later Put with lower ts. Set this to true to enable new semantics of versions. We
960     * will also consider mvcc in versions. See HBASE-15968 for details.
961     */
962    @Override
963    public boolean isNewVersionBehavior() {
964      return getStringOrDefault(NEW_VERSION_BEHAVIOR_BYTES, Boolean::parseBoolean,
965        DEFAULT_NEW_VERSION_BEHAVIOR);
966    }
967
968    public ModifyableColumnFamilyDescriptor setNewVersionBehavior(boolean newVersionBehavior) {
969      return setValue(NEW_VERSION_BEHAVIOR_BYTES, Boolean.toString(newVersionBehavior));
970    }
971
972    @Override
973    public int getTimeToLive() {
974      return getStringOrDefault(TTL_BYTES, Integer::parseInt, DEFAULT_TTL);
975    }
976
977    /**
978     * Set the time to live
979     * @param timeToLive Time-to-live of cell contents, in seconds.
980     * @return this (for chained invocation)
981     */
982    public ModifyableColumnFamilyDescriptor setTimeToLive(int timeToLive) {
983      return setValue(TTL_BYTES, Integer.toString(timeToLive));
984    }
985
986    /**
987     * Set the time to live
988     * @param timeToLive Time-to-live of cell contents, in seconds.
989     * @return this (for chained invocation)
990     * @throws org.apache.hadoop.hbase.exceptions.HBaseException exception
991     */
992    public ModifyableColumnFamilyDescriptor setTimeToLive(String timeToLive) throws HBaseException {
993      return setTimeToLive(Integer.parseInt(PrettyPrinter.valueOf(timeToLive, Unit.TIME_INTERVAL)));
994    }
995
996    @Override
997    public int getMinVersions() {
998      return getStringOrDefault(MIN_VERSIONS_BYTES, Integer::valueOf, DEFAULT_MIN_VERSIONS);
999    }
1000
1001    /**
1002     * Set minimum versions to retain.
1003     * @param minVersions The minimum number of versions to keep. (used when timeToLive is set)
1004     * @return this (for chained invocation)
1005     */
1006    public ModifyableColumnFamilyDescriptor setMinVersions(int minVersions) {
1007      return setValue(MIN_VERSIONS_BYTES, Integer.toString(minVersions));
1008    }
1009
1010    /**
1011     * Retain all versions for a given TTL(retentionInterval), and then only a specific number of
1012     * versions(versionAfterInterval) after that interval elapses.
1013     * @param retentionInterval    Retain all versions for this interval
1014     * @param versionAfterInterval Retain no of versions to retain after retentionInterval
1015     * @return this (for chained invocation)
1016     */
1017    public ModifyableColumnFamilyDescriptor setVersionsWithTimeToLive(final int retentionInterval,
1018      final int versionAfterInterval) {
1019      ModifyableColumnFamilyDescriptor modifyableColumnFamilyDescriptor =
1020        setVersions(versionAfterInterval, Integer.MAX_VALUE);
1021      modifyableColumnFamilyDescriptor.setTimeToLive(retentionInterval);
1022      modifyableColumnFamilyDescriptor.setKeepDeletedCells(KeepDeletedCells.TTL);
1023      return modifyableColumnFamilyDescriptor;
1024    }
1025
1026    @Override
1027    public boolean isBlockCacheEnabled() {
1028      return getStringOrDefault(BLOCKCACHE_BYTES, Boolean::valueOf, DEFAULT_BLOCKCACHE);
1029    }
1030
1031    /**
1032     * Set the blockCacheEnabled flag
1033     * @param blockCacheEnabled True if hfile DATA type blocks should be cached (We always cache
1034     *                          INDEX and BLOOM blocks; you cannot turn this off).
1035     * @return this (for chained invocation)
1036     */
1037    public ModifyableColumnFamilyDescriptor setBlockCacheEnabled(boolean blockCacheEnabled) {
1038      return setValue(BLOCKCACHE_BYTES, Boolean.toString(blockCacheEnabled));
1039    }
1040
1041    @Override
1042    public BloomType getBloomFilterType() {
1043      return getStringOrDefault(BLOOMFILTER_BYTES, n -> BloomType.valueOf(n.toUpperCase()),
1044        DEFAULT_BLOOMFILTER);
1045    }
1046
1047    public ModifyableColumnFamilyDescriptor setBloomFilterType(final BloomType bt) {
1048      return setValue(BLOOMFILTER_BYTES, bt.name());
1049    }
1050
1051    @Override
1052    public int getScope() {
1053      return getStringOrDefault(REPLICATION_SCOPE_BYTES, Integer::valueOf,
1054        DEFAULT_REPLICATION_SCOPE);
1055    }
1056
1057    public ModifyableColumnFamilyDescriptor setScope(int scope) {
1058      return setValue(REPLICATION_SCOPE_BYTES, Integer.toString(scope));
1059    }
1060
1061    @Override
1062    public boolean isCacheDataOnWrite() {
1063      return getStringOrDefault(CACHE_DATA_ON_WRITE_BYTES, Boolean::valueOf,
1064        DEFAULT_CACHE_DATA_ON_WRITE);
1065    }
1066
1067    /**
1068     * Set the setCacheDataOnWrite flag
1069     * @param value true if we should cache data blocks on write
1070     * @return this (for chained invocation)
1071     */
1072    public ModifyableColumnFamilyDescriptor setCacheDataOnWrite(boolean value) {
1073      return setValue(CACHE_DATA_ON_WRITE_BYTES, Boolean.toString(value));
1074    }
1075
1076    @Override
1077    public boolean isCacheIndexesOnWrite() {
1078      return getStringOrDefault(CACHE_INDEX_ON_WRITE_BYTES, Boolean::valueOf,
1079        DEFAULT_CACHE_INDEX_ON_WRITE);
1080    }
1081
1082    /**
1083     * Set the setCacheIndexesOnWrite flag
1084     * @param value true if we should cache index blocks on write
1085     * @return this (for chained invocation)
1086     */
1087    public ModifyableColumnFamilyDescriptor setCacheIndexesOnWrite(boolean value) {
1088      return setValue(CACHE_INDEX_ON_WRITE_BYTES, Boolean.toString(value));
1089    }
1090
1091    @Override
1092    public boolean isCacheBloomsOnWrite() {
1093      return getStringOrDefault(CACHE_BLOOMS_ON_WRITE_BYTES, Boolean::valueOf,
1094        DEFAULT_CACHE_BLOOMS_ON_WRITE);
1095    }
1096
1097    /**
1098     * Set the setCacheBloomsOnWrite flag.
1099     * @param value true if we should cache bloomfilter blocks on write
1100     * @return this (for chained invocation)
1101     */
1102    public ModifyableColumnFamilyDescriptor setCacheBloomsOnWrite(boolean value) {
1103      return setValue(CACHE_BLOOMS_ON_WRITE_BYTES, Boolean.toString(value));
1104    }
1105
1106    @Override
1107    public boolean isEvictBlocksOnClose() {
1108      return getStringOrDefault(EVICT_BLOCKS_ON_CLOSE_BYTES, Boolean::valueOf,
1109        DEFAULT_EVICT_BLOCKS_ON_CLOSE);
1110    }
1111
1112    /**
1113     * Set the setEvictBlocksOnClose flag.
1114     * @param value true if we should evict cached blocks from the blockcache on close
1115     * @return this (for chained invocation)
1116     */
1117    public ModifyableColumnFamilyDescriptor setEvictBlocksOnClose(boolean value) {
1118      return setValue(EVICT_BLOCKS_ON_CLOSE_BYTES, Boolean.toString(value));
1119    }
1120
1121    @Override
1122    public boolean isPrefetchBlocksOnOpen() {
1123      return getStringOrDefault(PREFETCH_BLOCKS_ON_OPEN_BYTES, Boolean::valueOf,
1124        DEFAULT_PREFETCH_BLOCKS_ON_OPEN);
1125    }
1126
1127    /**
1128     * Set the setPrefetchBlocksOnOpen flag
1129     * @param value true if we should prefetch blocks into the blockcache on open
1130     * @return this (for chained invocation)
1131     */
1132    public ModifyableColumnFamilyDescriptor setPrefetchBlocksOnOpen(boolean value) {
1133      return setValue(PREFETCH_BLOCKS_ON_OPEN_BYTES, Boolean.toString(value));
1134    }
1135
1136    @Override
1137    public String toString() {
1138      StringBuilder s = new StringBuilder();
1139      s.append('{');
1140      s.append(HConstants.NAME);
1141      s.append(" => '");
1142      s.append(getNameAsString());
1143      s.append("'");
1144      s.append(getValues(true));
1145      s.append('}');
1146      return s.toString();
1147    }
1148
1149    @Override
1150    public String toStringCustomizedValues() {
1151      StringBuilder s = new StringBuilder();
1152      s.append('{');
1153      s.append(HConstants.NAME);
1154      s.append(" => '");
1155      s.append(getNameAsString());
1156      s.append("'");
1157      s.append(getValues(false));
1158      s.append('}');
1159      return s.toString();
1160    }
1161
1162    private StringBuilder getValues(boolean printDefaults) {
1163      StringBuilder s = new StringBuilder();
1164
1165      boolean hasConfigKeys = false;
1166
1167      // print all reserved keys first
1168      for (Map.Entry<Bytes, Bytes> entry : values.entrySet()) {
1169        if (!RESERVED_KEYWORDS.contains(entry.getKey())) {
1170          hasConfigKeys = true;
1171          continue;
1172        }
1173        String key = Bytes.toString(entry.getKey().get());
1174        String value = Bytes.toStringBinary(entry.getValue().get());
1175        if (
1176          printDefaults || !DEFAULT_VALUES.containsKey(key)
1177            || !DEFAULT_VALUES.get(key).equalsIgnoreCase(value)
1178        ) {
1179          s.append(", ");
1180          s.append(key);
1181          s.append(" => ");
1182          s.append('\'').append(PrettyPrinter.format(value, getUnit(key))).append('\'');
1183        }
1184      }
1185
1186      // print all non-reserved, advanced config keys as a separate subset
1187      if (hasConfigKeys) {
1188        s.append(", ");
1189        s.append(HConstants.METADATA).append(" => ");
1190        s.append('{');
1191        boolean printComma = false;
1192        for (Map.Entry<Bytes, Bytes> entry : values.entrySet()) {
1193          Bytes k = entry.getKey();
1194          if (RESERVED_KEYWORDS.contains(k)) {
1195            continue;
1196          }
1197          String key = Bytes.toString(k.get());
1198          String value = Bytes.toStringBinary(entry.getValue().get());
1199          if (printComma) {
1200            s.append(", ");
1201          }
1202          printComma = true;
1203          s.append('\'').append(key).append('\'');
1204          s.append(" => ");
1205          s.append('\'').append(PrettyPrinter.format(value, getUnit(key))).append('\'');
1206        }
1207        s.append('}');
1208      }
1209
1210      if (!configuration.isEmpty()) {
1211        s.append(", ");
1212        s.append(HConstants.CONFIGURATION).append(" => ");
1213        s.append('{');
1214        boolean printCommaForConfiguration = false;
1215        for (Map.Entry<String, String> e : configuration.entrySet()) {
1216          if (printCommaForConfiguration) {
1217            s.append(", ");
1218          }
1219          printCommaForConfiguration = true;
1220          s.append('\'').append(e.getKey()).append('\'');
1221          s.append(" => ");
1222          s.append('\'').append(PrettyPrinter.format(e.getValue(), getUnit(e.getKey())))
1223            .append('\'');
1224        }
1225        s.append("}");
1226      }
1227      return s;
1228    }
1229
1230    @Override
1231    public boolean equals(Object obj) {
1232      if (this == obj) {
1233        return true;
1234      }
1235      if (obj instanceof ModifyableColumnFamilyDescriptor) {
1236        return ColumnFamilyDescriptor.COMPARATOR.compare(this,
1237          (ModifyableColumnFamilyDescriptor) obj) == 0;
1238      }
1239      return false;
1240    }
1241
1242    @Override
1243    public int hashCode() {
1244      int result = Bytes.hashCode(name);
1245      result ^= (int) COLUMN_DESCRIPTOR_VERSION;
1246      result ^= values.hashCode();
1247      result ^= configuration.hashCode();
1248      return result;
1249    }
1250
1251    @Override
1252    public int compareTo(ModifyableColumnFamilyDescriptor other) {
1253      return COMPARATOR.compare(this, other);
1254    }
1255
1256    /** Returns This instance serialized with pb with pb magic prefix */
1257    private byte[] toByteArray() {
1258      return ProtobufUtil.prependPBMagic(ProtobufUtil.toColumnFamilySchema(this).toByteArray());
1259    }
1260
1261    /**
1262     * Parse the serialized representation of a {@link ModifyableColumnFamilyDescriptor}
1263     * @param bytes A pb serialized {@link ModifyableColumnFamilyDescriptor} instance with pb magic
1264     *              prefix
1265     * @return An instance of {@link ModifyableColumnFamilyDescriptor} made from <code>bytes</code>
1266     * @see #toByteArray()
1267     */
1268    private static ColumnFamilyDescriptor parseFrom(final byte[] bytes)
1269      throws DeserializationException {
1270      if (!ProtobufUtil.isPBMagicPrefix(bytes)) {
1271        throw new DeserializationException("No magic");
1272      }
1273      int pblen = ProtobufUtil.lengthOfPBMagic();
1274      ColumnFamilySchema.Builder builder = ColumnFamilySchema.newBuilder();
1275      ColumnFamilySchema cfs = null;
1276      try {
1277        ProtobufUtil.mergeFrom(builder, bytes, pblen, bytes.length - pblen);
1278        cfs = builder.build();
1279      } catch (IOException e) {
1280        throw new DeserializationException(e);
1281      }
1282      return ProtobufUtil.toColumnFamilyDescriptor(cfs);
1283    }
1284
1285    @Override
1286    public String getConfigurationValue(String key) {
1287      return configuration.get(key);
1288    }
1289
1290    @Override
1291    public Map<String, String> getConfiguration() {
1292      // shallow pointer copy
1293      return Collections.unmodifiableMap(configuration);
1294    }
1295
1296    /**
1297     * Setter for storing a configuration setting in {@link #configuration} map.
1298     * @param key   Config key. Same as XML config key e.g. hbase.something.or.other.
1299     * @param value String value. If null, removes the configuration.
1300     * @return this (for chained invocation)
1301     */
1302    public ModifyableColumnFamilyDescriptor setConfiguration(String key, String value) {
1303      if (value == null || value.length() == 0) {
1304        configuration.remove(key);
1305      } else {
1306        configuration.put(key, value);
1307      }
1308      return this;
1309    }
1310
1311    /**
1312     * Remove a configuration setting represented by the key from the {@link #configuration} map.
1313     * @return this (for chained invocation)
1314     */
1315    public ModifyableColumnFamilyDescriptor removeConfiguration(final String key) {
1316      return setConfiguration(key, null);
1317    }
1318
1319    @Override
1320    public String getEncryptionType() {
1321      return getStringOrDefault(ENCRYPTION_BYTES, Function.identity(), null);
1322    }
1323
1324    /**
1325     * Set the encryption algorithm for use with this family
1326     * @return this (for chained invocation)
1327     */
1328    public ModifyableColumnFamilyDescriptor setEncryptionType(String algorithm) {
1329      return setValue(ENCRYPTION_BYTES, algorithm);
1330    }
1331
1332    @Override
1333    public byte[] getEncryptionKey() {
1334      return getOrDefault(ENCRYPTION_KEY_BYTES, Bytes::copy, null);
1335    }
1336
1337    /**
1338     * Set the raw crypto key attribute for the family
1339     * @return this (for chained invocation)
1340     */
1341    public ModifyableColumnFamilyDescriptor setEncryptionKey(byte[] keyBytes) {
1342      return setValue(ENCRYPTION_KEY_BYTES, new Bytes(keyBytes));
1343    }
1344
1345    @Override
1346    public long getMobThreshold() {
1347      return getStringOrDefault(MOB_THRESHOLD_BYTES, Long::valueOf, DEFAULT_MOB_THRESHOLD);
1348    }
1349
1350    /**
1351     * Sets the mob threshold of the family.
1352     * @param threshold The mob threshold.
1353     * @return this (for chained invocation)
1354     */
1355    public ModifyableColumnFamilyDescriptor setMobThreshold(long threshold) {
1356      return setValue(MOB_THRESHOLD_BYTES, String.valueOf(threshold));
1357    }
1358
1359    @Override
1360    public boolean isMobEnabled() {
1361      return getStringOrDefault(IS_MOB_BYTES, Boolean::valueOf, DEFAULT_MOB);
1362    }
1363
1364    /**
1365     * Enables the mob for the family.
1366     * @param isMobEnabled Whether to enable the mob for the family.
1367     * @return this (for chained invocation)
1368     */
1369    public ModifyableColumnFamilyDescriptor setMobEnabled(boolean isMobEnabled) {
1370      return setValue(IS_MOB_BYTES, String.valueOf(isMobEnabled));
1371    }
1372
1373    @Override
1374    public MobCompactPartitionPolicy getMobCompactPartitionPolicy() {
1375      return getStringOrDefault(MOB_COMPACT_PARTITION_POLICY_BYTES,
1376        n -> MobCompactPartitionPolicy.valueOf(n.toUpperCase()),
1377        DEFAULT_MOB_COMPACT_PARTITION_POLICY);
1378    }
1379
1380    /**
1381     * Set the mob compact partition policy for the family.
1382     * @param policy policy type
1383     * @return this (for chained invocation)
1384     */
1385    public ModifyableColumnFamilyDescriptor
1386      setMobCompactPartitionPolicy(MobCompactPartitionPolicy policy) {
1387      return setValue(MOB_COMPACT_PARTITION_POLICY_BYTES, policy.name());
1388    }
1389
1390    @Override
1391    public short getDFSReplication() {
1392      return getStringOrDefault(DFS_REPLICATION_BYTES, Short::valueOf, DEFAULT_DFS_REPLICATION);
1393    }
1394
1395    /**
1396     * Set the replication factor to hfile(s) belonging to this family
1397     * @param replication number of replicas the blocks(s) belonging to this CF should have, or
1398     *                    {@link #DEFAULT_DFS_REPLICATION} for the default replication factor set in
1399     *                    the filesystem
1400     * @return this (for chained invocation)
1401     */
1402    public ModifyableColumnFamilyDescriptor setDFSReplication(short replication) {
1403      if (replication < 1 && replication != DEFAULT_DFS_REPLICATION) {
1404        throw new IllegalArgumentException(
1405          "DFS replication factor cannot be less than 1 if explicitly set.");
1406      }
1407      return setValue(DFS_REPLICATION_BYTES, Short.toString(replication));
1408    }
1409
1410    @Override
1411    public String getStoragePolicy() {
1412      return getStringOrDefault(STORAGE_POLICY_BYTES, Function.identity(), null);
1413    }
1414
1415    /**
1416     * Set the storage policy for use with this family
1417     * @param policy the policy to set, valid setting includes: <i>"LAZY_PERSIST"</i>,
1418     *               <i>"ALL_SSD"</i>, <i>"ONE_SSD"</i>, <i>"HOT"</i>, <i>"WARM"</i>, <i>"COLD"</i>
1419     * @return this (for chained invocation)
1420     */
1421    public ModifyableColumnFamilyDescriptor setStoragePolicy(String policy) {
1422      return setValue(STORAGE_POLICY_BYTES, policy);
1423    }
1424
1425  }
1426}