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