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