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