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