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