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.ArrayList;
022import java.util.Arrays;
023import java.util.Collection;
024import java.util.Collections;
025import java.util.HashMap;
026import java.util.HashSet;
027import java.util.List;
028import java.util.Map;
029import java.util.Objects;
030import java.util.Optional;
031import java.util.Set;
032import java.util.TreeMap;
033import java.util.TreeSet;
034import java.util.function.BiPredicate;
035import java.util.function.Function;
036import java.util.regex.Matcher;
037import java.util.regex.Pattern;
038import java.util.stream.Collectors;
039import org.apache.hadoop.hbase.Coprocessor;
040import org.apache.hadoop.hbase.HConstants;
041import org.apache.hadoop.hbase.TableName;
042import org.apache.hadoop.hbase.exceptions.DeserializationException;
043import org.apache.hadoop.hbase.exceptions.HBaseException;
044import org.apache.hadoop.hbase.rsgroup.RSGroupInfo;
045import org.apache.hadoop.hbase.util.Bytes;
046import org.apache.hadoop.hbase.util.PrettyPrinter;
047import org.apache.yetus.audience.InterfaceAudience;
048import org.slf4j.Logger;
049import org.slf4j.LoggerFactory;
050
051import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
052import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
053
054/**
055 * Convenience class for composing an instance of {@link TableDescriptor}.
056 * @since 2.0.0
057 */
058@InterfaceAudience.Public
059public class TableDescriptorBuilder {
060  public static final Logger LOG = LoggerFactory.getLogger(TableDescriptorBuilder.class);
061  @InterfaceAudience.Private
062  public static final String SPLIT_POLICY = "SPLIT_POLICY";
063  private static final Bytes SPLIT_POLICY_KEY = new Bytes(Bytes.toBytes(SPLIT_POLICY));
064  /**
065   * Used by HBase Shell interface to access this metadata attribute which denotes the maximum size
066   * of the store file after which a region split occurs.
067   */
068  @InterfaceAudience.Private
069  public static final String MAX_FILESIZE = "MAX_FILESIZE";
070  private static final Bytes MAX_FILESIZE_KEY = new Bytes(Bytes.toBytes(MAX_FILESIZE));
071
072  /**
073   * Used by rest interface to access this metadata attribute which denotes if the table is Read
074   * Only.
075   */
076  @InterfaceAudience.Private
077  public static final String READONLY = "READONLY";
078  private static final Bytes READONLY_KEY = new Bytes(Bytes.toBytes(READONLY));
079
080  /**
081   * Used by HBase Shell interface to access this metadata attribute which denotes if the table is
082   * compaction enabled.
083   */
084  @InterfaceAudience.Private
085  public static final String COMPACTION_ENABLED = "COMPACTION_ENABLED";
086  private static final Bytes COMPACTION_ENABLED_KEY = new Bytes(Bytes.toBytes(COMPACTION_ENABLED));
087
088  /**
089   * Used by HBase Shell interface to access this metadata attribute which denotes if the table is
090   * split enabled.
091   */
092  @InterfaceAudience.Private
093  public static final String SPLIT_ENABLED = "SPLIT_ENABLED";
094  private static final Bytes SPLIT_ENABLED_KEY = new Bytes(Bytes.toBytes(SPLIT_ENABLED));
095
096  /**
097   * Used by HBase Shell interface to access this metadata attribute which denotes if the table is
098   * merge enabled.
099   */
100  @InterfaceAudience.Private
101  public static final String MERGE_ENABLED = "MERGE_ENABLED";
102  private static final Bytes MERGE_ENABLED_KEY = new Bytes(Bytes.toBytes(MERGE_ENABLED));
103
104  /**
105   * Used by HBase Shell interface to access this metadata attribute which represents the maximum
106   * size of the memstore after which its contents are flushed onto the disk.
107   */
108  @InterfaceAudience.Private
109  public static final String MEMSTORE_FLUSHSIZE = "MEMSTORE_FLUSHSIZE";
110  private static final Bytes MEMSTORE_FLUSHSIZE_KEY = new Bytes(Bytes.toBytes(MEMSTORE_FLUSHSIZE));
111
112  @InterfaceAudience.Private
113  public static final String FLUSH_POLICY = "FLUSH_POLICY";
114  private static final Bytes FLUSH_POLICY_KEY = new Bytes(Bytes.toBytes(FLUSH_POLICY));
115  /**
116   * Used by rest interface to access this metadata attribute which denotes if it is a catalog
117   * table, either <code> hbase:meta </code>.
118   */
119  @InterfaceAudience.Private
120  public static final String IS_META = "IS_META";
121  private static final Bytes IS_META_KEY = new Bytes(Bytes.toBytes(IS_META));
122
123  /**
124   * {@link Durability} setting for the table.
125   */
126  @InterfaceAudience.Private
127  public static final String DURABILITY = "DURABILITY";
128  private static final Bytes DURABILITY_KEY = new Bytes(Bytes.toBytes("DURABILITY"));
129
130  /**
131   * The number of region replicas for the table.
132   */
133  @InterfaceAudience.Private
134  public static final String REGION_REPLICATION = "REGION_REPLICATION";
135  private static final Bytes REGION_REPLICATION_KEY = new Bytes(Bytes.toBytes(REGION_REPLICATION));
136
137  /**
138   * The flag to indicate whether or not the memstore should be replicated for read-replicas
139   * (CONSISTENCY =&gt; TIMELINE).
140   */
141  @InterfaceAudience.Private
142  public static final String REGION_MEMSTORE_REPLICATION = "REGION_MEMSTORE_REPLICATION";
143  private static final Bytes REGION_MEMSTORE_REPLICATION_KEY =
144    new Bytes(Bytes.toBytes(REGION_MEMSTORE_REPLICATION));
145
146  /**
147   * If non-null, the HDFS erasure coding policy to set on the data dir of the table
148   */
149  public static final String ERASURE_CODING_POLICY = "ERASURE_CODING_POLICY";
150  private static final Bytes ERASURE_CODING_POLICY_KEY =
151    new Bytes(Bytes.toBytes(ERASURE_CODING_POLICY));
152
153  private static final String DEFAULT_ERASURE_CODING_POLICY = null;
154  /**
155   * Used by shell/rest interface to access this metadata attribute which denotes if the table
156   * should be treated by region normalizer.
157   */
158  @InterfaceAudience.Private
159  public static final String NORMALIZATION_ENABLED = "NORMALIZATION_ENABLED";
160  private static final Bytes NORMALIZATION_ENABLED_KEY =
161    new Bytes(Bytes.toBytes(NORMALIZATION_ENABLED));
162
163  @InterfaceAudience.Private
164  public static final String NORMALIZER_TARGET_REGION_COUNT = "NORMALIZER_TARGET_REGION_COUNT";
165  private static final Bytes NORMALIZER_TARGET_REGION_COUNT_KEY =
166    new Bytes(Bytes.toBytes(NORMALIZER_TARGET_REGION_COUNT));
167
168  @InterfaceAudience.Private
169  public static final String NORMALIZER_TARGET_REGION_SIZE_MB = "NORMALIZER_TARGET_REGION_SIZE_MB";
170  private static final Bytes NORMALIZER_TARGET_REGION_SIZE_MB_KEY =
171    new Bytes(Bytes.toBytes(NORMALIZER_TARGET_REGION_SIZE_MB));
172  // TODO: Keeping backward compatability with HBASE-25651 change. Can be removed in later version
173  @InterfaceAudience.Private
174  @Deprecated
175  public static final String NORMALIZER_TARGET_REGION_SIZE = "NORMALIZER_TARGET_REGION_SIZE";
176  @Deprecated
177  private static final Bytes NORMALIZER_TARGET_REGION_SIZE_KEY =
178    new Bytes(Bytes.toBytes(NORMALIZER_TARGET_REGION_SIZE));
179
180  /**
181   * Default durability for HTD is USE_DEFAULT, which defaults to HBase-global default value
182   */
183  private static final Durability DEFAULT_DURABLITY = Durability.USE_DEFAULT;
184
185  @InterfaceAudience.Private
186  public static final String PRIORITY = "PRIORITY";
187  private static final Bytes PRIORITY_KEY = new Bytes(Bytes.toBytes(PRIORITY));
188
189  private static final Bytes RSGROUP_KEY =
190    new Bytes(Bytes.toBytes(RSGroupInfo.TABLE_DESC_PROP_GROUP));
191
192  /**
193   * Relative priority of the table used for rpc scheduling
194   */
195  private static final int DEFAULT_PRIORITY = HConstants.NORMAL_QOS;
196
197  /**
198   * Constant that denotes whether the table is READONLY by default and is false
199   */
200  public static final boolean DEFAULT_READONLY = false;
201
202  /**
203   * Constant that denotes whether the table is compaction enabled by default
204   */
205  public static final boolean DEFAULT_COMPACTION_ENABLED = true;
206
207  /**
208   * Constant that denotes whether the table is split enabled by default
209   */
210  public static final boolean DEFAULT_SPLIT_ENABLED = true;
211
212  /**
213   * Constant that denotes whether the table is merge enabled by default
214   */
215  public static final boolean DEFAULT_MERGE_ENABLED = true;
216
217  /**
218   * Constant that denotes the maximum default size of the memstore in bytes after which the
219   * contents are flushed to the store files.
220   */
221  public static final long DEFAULT_MEMSTORE_FLUSH_SIZE = 1024 * 1024 * 128L;
222
223  public static final int DEFAULT_REGION_REPLICATION = 1;
224
225  public static final boolean DEFAULT_REGION_MEMSTORE_REPLICATION = true;
226
227  private final static Map<String, String> DEFAULT_VALUES = new HashMap<>();
228  private final static Set<Bytes> RESERVED_KEYWORDS = new HashSet<>();
229
230  static {
231    DEFAULT_VALUES.put(MAX_FILESIZE, String.valueOf(HConstants.DEFAULT_MAX_FILE_SIZE));
232    DEFAULT_VALUES.put(READONLY, String.valueOf(DEFAULT_READONLY));
233    DEFAULT_VALUES.put(MEMSTORE_FLUSHSIZE, String.valueOf(DEFAULT_MEMSTORE_FLUSH_SIZE));
234    DEFAULT_VALUES.put(DURABILITY, DEFAULT_DURABLITY.name()); // use the enum name
235    DEFAULT_VALUES.put(REGION_REPLICATION, String.valueOf(DEFAULT_REGION_REPLICATION));
236    DEFAULT_VALUES.put(PRIORITY, String.valueOf(DEFAULT_PRIORITY));
237    // Setting ERASURE_CODING_POLICY to NULL so that it is not considered as metadata
238    DEFAULT_VALUES.put(ERASURE_CODING_POLICY, String.valueOf(DEFAULT_ERASURE_CODING_POLICY));
239    DEFAULT_VALUES.keySet().stream().map(s -> new Bytes(Bytes.toBytes(s)))
240      .forEach(RESERVED_KEYWORDS::add);
241    RESERVED_KEYWORDS.add(IS_META_KEY);
242  }
243
244  public static PrettyPrinter.Unit getUnit(String key) {
245    switch (key) {
246      case MAX_FILESIZE:
247      case MEMSTORE_FLUSHSIZE:
248        return PrettyPrinter.Unit.BYTE;
249      default:
250        return PrettyPrinter.Unit.NONE;
251    }
252  }
253
254  /**
255   * @deprecated namespace table has been folded into the ns family in meta table, do not use this
256   *             any more.
257   */
258  @InterfaceAudience.Private
259  @Deprecated
260  public final static String NAMESPACE_FAMILY_INFO = "info";
261
262  /**
263   * @deprecated namespace table has been folded into the ns family in meta table, do not use this
264   *             any more.
265   */
266  @InterfaceAudience.Private
267  @Deprecated
268  public final static byte[] NAMESPACE_FAMILY_INFO_BYTES = Bytes.toBytes(NAMESPACE_FAMILY_INFO);
269
270  /**
271   * @deprecated namespace table has been folded into the ns family in meta table, do not use this
272   *             any more.
273   */
274  @InterfaceAudience.Private
275  @Deprecated
276  public final static byte[] NAMESPACE_COL_DESC_BYTES = Bytes.toBytes("d");
277
278  /**
279   * <pre>
280   * Pattern that matches a coprocessor specification. Form is:
281   * {@code <coprocessor jar file location> '|' <class name> ['|' <priority> ['|' <arguments>]]}
282   * where arguments are {@code <KEY> '=' <VALUE> [,...]}
283   * For example: {@code hdfs:///foo.jar|com.foo.FooRegionObserver|1001|arg1=1,arg2=2}
284   * </pre>
285   */
286  private static final Pattern CP_HTD_ATTR_VALUE_PATTERN =
287    Pattern.compile("(^[^\\|]*)\\|([^\\|]+)\\|[\\s]*([\\d]*)[\\s]*(\\|.*)?$");
288
289  private static final String CP_HTD_ATTR_VALUE_PARAM_KEY_PATTERN = "[^=,]+";
290  private static final String CP_HTD_ATTR_VALUE_PARAM_VALUE_PATTERN = "[^,]+";
291  private static final Pattern CP_HTD_ATTR_VALUE_PARAM_PATTERN = Pattern.compile("("
292    + CP_HTD_ATTR_VALUE_PARAM_KEY_PATTERN + ")=(" + CP_HTD_ATTR_VALUE_PARAM_VALUE_PATTERN + "),?");
293  private static final Pattern CP_HTD_ATTR_KEY_PATTERN =
294    Pattern.compile("^coprocessor\\$([0-9]+)$", Pattern.CASE_INSENSITIVE);
295
296  /**
297   * Table descriptor for namespace table
298   * @deprecated since 3.0.0 and will be removed in 4.0.0. We have folded the data in namespace
299   *             table into meta table, so do not use it any more.
300   * @see <a href="https://issues.apache.org/jira/browse/HBASE-21154">HBASE-21154</a>
301   */
302  @Deprecated
303  public static final TableDescriptor NAMESPACE_TABLEDESC =
304    TableDescriptorBuilder.newBuilder(TableName.NAMESPACE_TABLE_NAME)
305      .setColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(NAMESPACE_FAMILY_INFO_BYTES)
306        // Ten is arbitrary number. Keep versions to help debugging.
307        .setMaxVersions(10).setInMemory(true).setBlocksize(8 * 1024)
308        .setScope(HConstants.REPLICATION_SCOPE_LOCAL).build())
309      .build();
310
311  private final ModifyableTableDescriptor desc;
312
313  /** Returns This instance serialized with pb with pb magic prefix */
314  public static byte[] toByteArray(TableDescriptor desc) {
315    if (desc instanceof ModifyableTableDescriptor) {
316      return ((ModifyableTableDescriptor) desc).toByteArray();
317    }
318    return new ModifyableTableDescriptor(desc).toByteArray();
319  }
320
321  /**
322   * The input should be created by {@link #toByteArray}.
323   * @param pbBytes A pb serialized TableDescriptor instance with pb magic prefix
324   * @return This instance serialized with pb with pb magic prefix
325   * @throws org.apache.hadoop.hbase.exceptions.DeserializationException if an error occurred
326   */
327  public static TableDescriptor parseFrom(byte[] pbBytes) throws DeserializationException {
328    return ModifyableTableDescriptor.parseFrom(pbBytes);
329  }
330
331  public static TableDescriptorBuilder newBuilder(final TableName name) {
332    return new TableDescriptorBuilder(name);
333  }
334
335  public static TableDescriptor copy(TableDescriptor desc) {
336    return new ModifyableTableDescriptor(desc);
337  }
338
339  public static TableDescriptor copy(TableName name, TableDescriptor desc) {
340    return new ModifyableTableDescriptor(name, desc);
341  }
342
343  /**
344   * Copy all values, families, and name from the input.
345   * @param desc The desciptor to copy
346   * @return A clone of input
347   */
348  public static TableDescriptorBuilder newBuilder(final TableDescriptor desc) {
349    return new TableDescriptorBuilder(desc);
350  }
351
352  private TableDescriptorBuilder(final TableName name) {
353    this.desc = new ModifyableTableDescriptor(name);
354  }
355
356  private TableDescriptorBuilder(final TableDescriptor desc) {
357    this.desc = new ModifyableTableDescriptor(desc);
358  }
359
360  public TableDescriptorBuilder setCoprocessor(String className) throws IOException {
361    return setCoprocessor(CoprocessorDescriptorBuilder.of(className));
362  }
363
364  public TableDescriptorBuilder setCoprocessor(CoprocessorDescriptor cpDesc) throws IOException {
365    desc.setCoprocessor(Objects.requireNonNull(cpDesc));
366    return this;
367  }
368
369  public TableDescriptorBuilder setCoprocessors(Collection<CoprocessorDescriptor> cpDescs)
370    throws IOException {
371    for (CoprocessorDescriptor cpDesc : cpDescs) {
372      desc.setCoprocessor(cpDesc);
373    }
374    return this;
375  }
376
377  public boolean hasCoprocessor(String classNameToMatch) {
378    return desc.hasCoprocessor(classNameToMatch);
379  }
380
381  public TableDescriptorBuilder setColumnFamily(final ColumnFamilyDescriptor family) {
382    desc.setColumnFamily(Objects.requireNonNull(family));
383    return this;
384  }
385
386  public TableDescriptorBuilder
387    setColumnFamilies(final Collection<ColumnFamilyDescriptor> families) {
388    families.forEach(desc::setColumnFamily);
389    return this;
390  }
391
392  public TableDescriptorBuilder modifyColumnFamily(final ColumnFamilyDescriptor family) {
393    desc.modifyColumnFamily(Objects.requireNonNull(family));
394    return this;
395  }
396
397  public TableDescriptorBuilder removeValue(final String key) {
398    desc.removeValue(key);
399    return this;
400  }
401
402  public TableDescriptorBuilder removeValue(Bytes key) {
403    desc.removeValue(key);
404    return this;
405  }
406
407  public TableDescriptorBuilder removeValue(byte[] key) {
408    desc.removeValue(key);
409    return this;
410  }
411
412  public TableDescriptorBuilder removeValue(BiPredicate<Bytes, Bytes> predicate) {
413    List<Bytes> toRemove =
414      desc.getValues().entrySet().stream().filter(e -> predicate.test(e.getKey(), e.getValue()))
415        .map(Map.Entry::getKey).collect(Collectors.toList());
416    for (Bytes key : toRemove) {
417      removeValue(key);
418    }
419    return this;
420  }
421
422  public TableDescriptorBuilder removeColumnFamily(final byte[] name) {
423    desc.removeColumnFamily(name);
424    return this;
425  }
426
427  public TableDescriptorBuilder removeCoprocessor(String className) {
428    desc.removeCoprocessor(className);
429    return this;
430  }
431
432  public TableDescriptorBuilder setCompactionEnabled(final boolean isEnable) {
433    desc.setCompactionEnabled(isEnable);
434    return this;
435  }
436
437  public TableDescriptorBuilder setSplitEnabled(final boolean isEnable) {
438    desc.setSplitEnabled(isEnable);
439    return this;
440  }
441
442  public TableDescriptorBuilder setMergeEnabled(final boolean isEnable) {
443    desc.setMergeEnabled(isEnable);
444    return this;
445  }
446
447  public TableDescriptorBuilder setDurability(Durability durability) {
448    desc.setDurability(durability);
449    return this;
450  }
451
452  public TableDescriptorBuilder setFlushPolicyClassName(String clazz) {
453    desc.setFlushPolicyClassName(clazz);
454    return this;
455  }
456
457  public TableDescriptorBuilder setMaxFileSize(long maxFileSize) {
458    desc.setMaxFileSize(maxFileSize);
459    return this;
460  }
461
462  public TableDescriptorBuilder setMaxFileSize(String maxFileSize) throws HBaseException {
463    desc.setMaxFileSize(maxFileSize);
464    return this;
465  }
466
467  public TableDescriptorBuilder setMemStoreFlushSize(long memstoreFlushSize) {
468    desc.setMemStoreFlushSize(memstoreFlushSize);
469    return this;
470  }
471
472  public TableDescriptorBuilder setMemStoreFlushSize(String memStoreFlushSize)
473    throws HBaseException {
474    desc.setMemStoreFlushSize(memStoreFlushSize);
475    return this;
476  }
477
478  public TableDescriptorBuilder setNormalizerTargetRegionCount(final int regionCount) {
479    desc.setNormalizerTargetRegionCount(regionCount);
480    return this;
481  }
482
483  public TableDescriptorBuilder setNormalizerTargetRegionSize(final long regionSize) {
484    desc.setNormalizerTargetRegionSize(regionSize);
485    return this;
486  }
487
488  public TableDescriptorBuilder setNormalizationEnabled(final boolean isEnable) {
489    desc.setNormalizationEnabled(isEnable);
490    return this;
491  }
492
493  public TableDescriptorBuilder setPriority(int priority) {
494    desc.setPriority(priority);
495    return this;
496  }
497
498  public TableDescriptorBuilder setReadOnly(final boolean readOnly) {
499    desc.setReadOnly(readOnly);
500    return this;
501  }
502
503  public TableDescriptorBuilder setErasureCodingPolicy(String policy) {
504    desc.setErasureCodingPolicy(policy);
505    return this;
506  }
507
508  public TableDescriptorBuilder setRegionMemStoreReplication(boolean memstoreReplication) {
509    desc.setRegionMemStoreReplication(memstoreReplication);
510    return this;
511  }
512
513  public TableDescriptorBuilder setRegionReplication(int regionReplication) {
514    desc.setRegionReplication(regionReplication);
515    return this;
516  }
517
518  public TableDescriptorBuilder setRegionSplitPolicyClassName(String clazz) {
519    desc.setRegionSplitPolicyClassName(clazz);
520    return this;
521  }
522
523  public TableDescriptorBuilder setValue(final String key, final String value) {
524    desc.setValue(key, value);
525    return this;
526  }
527
528  public TableDescriptorBuilder setValue(final Bytes key, final Bytes value) {
529    desc.setValue(key, value);
530    return this;
531  }
532
533  public TableDescriptorBuilder setValue(final byte[] key, final byte[] value) {
534    desc.setValue(key, value);
535    return this;
536  }
537
538  public String getValue(String key) {
539    return desc.getValue(key);
540  }
541
542  /**
543   * Sets replication scope all & only the columns already in the builder. Columns added later won't
544   * be backfilled with replication scope.
545   * @param scope replication scope
546   * @return a TableDescriptorBuilder
547   */
548  public TableDescriptorBuilder setReplicationScope(int scope) {
549    Map<byte[], ColumnFamilyDescriptor> newFamilies = new TreeMap<>(Bytes.BYTES_RAWCOMPARATOR);
550    newFamilies.putAll(desc.families);
551    newFamilies.forEach((cf, cfDesc) -> {
552      desc.removeColumnFamily(cf);
553      desc
554        .setColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(cfDesc).setScope(scope).build());
555    });
556    return this;
557  }
558
559  public TableDescriptorBuilder setRegionServerGroup(String group) {
560    desc.setValue(RSGROUP_KEY, group);
561    return this;
562  }
563
564  public TableDescriptor build() {
565    return new ModifyableTableDescriptor(desc);
566  }
567
568  private static final class ModifyableTableDescriptor
569    implements TableDescriptor, Comparable<ModifyableTableDescriptor> {
570
571    private final TableName name;
572
573    /**
574     * A map which holds the metadata information of the table. This metadata includes values like
575     * IS_META, SPLIT_POLICY, MAX_FILE_SIZE, READONLY, MEMSTORE_FLUSHSIZE etc...
576     */
577    private final Map<Bytes, Bytes> values = new HashMap<>();
578
579    /**
580     * Maps column family name to the respective FamilyDescriptors
581     */
582    private final Map<byte[], ColumnFamilyDescriptor> families =
583      new TreeMap<>(Bytes.BYTES_RAWCOMPARATOR);
584
585    /**
586     * Construct a table descriptor specifying a TableName object
587     * @param name Table name.
588     */
589    private ModifyableTableDescriptor(final TableName name) {
590      this(name, Collections.emptyList(), Collections.emptyMap());
591    }
592
593    private ModifyableTableDescriptor(final TableDescriptor desc) {
594      this(desc.getTableName(), Arrays.asList(desc.getColumnFamilies()), desc.getValues());
595    }
596
597    /**
598     * Construct a table descriptor by cloning the descriptor passed as a parameter.
599     * <p>
600     * Makes a deep copy of the supplied descriptor.
601     * @param name The new name
602     * @param desc The descriptor.
603     */
604    private ModifyableTableDescriptor(final TableName name, final TableDescriptor desc) {
605      this(name, Arrays.asList(desc.getColumnFamilies()), desc.getValues());
606    }
607
608    private ModifyableTableDescriptor(final TableName name,
609      final Collection<ColumnFamilyDescriptor> families, Map<Bytes, Bytes> values) {
610      this.name = name;
611      families.forEach(c -> this.families.put(c.getName(), ColumnFamilyDescriptorBuilder.copy(c)));
612      this.values.putAll(values);
613      this.values.put(IS_META_KEY,
614        new Bytes(Bytes.toBytes(Boolean.toString(name.equals(TableName.META_TABLE_NAME)))));
615    }
616
617    /**
618     * Checks if this table is <code> hbase:meta </code> region.
619     * @return true if this table is <code> hbase:meta </code> region
620     */
621    @Override
622    public boolean isMetaRegion() {
623      return getOrDefault(IS_META_KEY, Boolean::valueOf, false);
624    }
625
626    /**
627     * Checks if the table is a <code>hbase:meta</code> table
628     * @return true if table is <code> hbase:meta </code> region.
629     */
630    @Override
631    public boolean isMetaTable() {
632      return isMetaRegion();
633    }
634
635    @Override
636    public Bytes getValue(Bytes key) {
637      Bytes rval = values.get(key);
638      return rval == null ? null : new Bytes(rval.copyBytes());
639    }
640
641    @Override
642    public String getValue(String key) {
643      Bytes rval = values.get(new Bytes(Bytes.toBytes(key)));
644      return rval == null ? null : Bytes.toString(rval.get(), rval.getOffset(), rval.getLength());
645    }
646
647    @Override
648    public byte[] getValue(byte[] key) {
649      Bytes value = values.get(new Bytes(key));
650      return value == null ? null : value.copyBytes();
651    }
652
653    private <T> T getOrDefault(Bytes key, Function<String, T> function, T defaultValue) {
654      Bytes value = values.get(key);
655      if (value == null) {
656        return defaultValue;
657      } else {
658        return function.apply(Bytes.toString(value.get(), value.getOffset(), value.getLength()));
659      }
660    }
661
662    /**
663     * Getter for fetching an unmodifiable {@link #values} map.
664     * @return unmodifiable map {@link #values}.
665     * @see #values
666     */
667    @Override
668    public Map<Bytes, Bytes> getValues() {
669      // shallow pointer copy
670      return Collections.unmodifiableMap(values);
671    }
672
673    /**
674     * Setter for storing metadata as a (key, value) pair in {@link #values} map
675     * @param key   The key.
676     * @param value The value. If null, removes the setting.
677     * @return the modifyable TD
678     * @see #values
679     */
680    public ModifyableTableDescriptor setValue(byte[] key, byte[] value) {
681      return setValue(toBytesOrNull(key, v -> v), toBytesOrNull(value, v -> v));
682    }
683
684    public ModifyableTableDescriptor setValue(String key, String value) {
685      return setValue(toBytesOrNull(key, Bytes::toBytes), toBytesOrNull(value, Bytes::toBytes));
686    }
687
688    /**
689     * @param key   The key.
690     * @param value The value. If null, removes the setting.
691     */
692    private ModifyableTableDescriptor setValue(final Bytes key, final String value) {
693      return setValue(key, toBytesOrNull(value, Bytes::toBytes));
694    }
695
696    /**
697     * Setter for storing metadata as a (key, value) pair in {@link #values} map
698     * @param key   The key.
699     * @param value The value. If null, removes the setting.
700     */
701    public ModifyableTableDescriptor setValue(final Bytes key, final Bytes value) {
702      if (value == null || value.getLength() == 0) {
703        values.remove(key);
704      } else {
705        values.put(key, value);
706      }
707      return this;
708    }
709
710    private static <T> Bytes toBytesOrNull(T t, Function<T, byte[]> f) {
711      if (t == null) {
712        return null;
713      } else {
714        return new Bytes(f.apply(t));
715      }
716    }
717
718    /**
719     * Remove metadata represented by the key from the {@link #values} map
720     * @param key Key whose key and value we're to remove from TableDescriptor parameters.
721     * @return the modifyable TD
722     */
723    public ModifyableTableDescriptor removeValue(final String key) {
724      return setValue(key, (String) null);
725    }
726
727    /**
728     * Remove metadata represented by the key from the {@link #values} map
729     * @param key Key whose key and value we're to remove from TableDescriptor parameters.
730     * @return the modifyable TD
731     */
732    public ModifyableTableDescriptor removeValue(Bytes key) {
733      return setValue(key, (Bytes) null);
734    }
735
736    /**
737     * Remove metadata represented by the key from the {@link #values} map
738     * @param key Key whose key and value we're to remove from TableDescriptor parameters.
739     * @return the modifyable TD
740     */
741    public ModifyableTableDescriptor removeValue(final byte[] key) {
742      return removeValue(new Bytes(key));
743    }
744
745    /**
746     * Check if the readOnly flag of the table is set. If the readOnly flag is set then the contents
747     * of the table can only be read from but not modified.
748     * @return true if all columns in the table should be read only
749     */
750    @Override
751    public boolean isReadOnly() {
752      return getOrDefault(READONLY_KEY, Boolean::valueOf, DEFAULT_READONLY);
753    }
754
755    /**
756     * Setting the table as read only sets all the columns in the table as read only. By default all
757     * tables are modifiable, but if the readOnly flag is set to true then the contents of the table
758     * can only be read but not modified.
759     * @param readOnly True if all of the columns in the table should be read only.
760     * @return the modifyable TD
761     */
762    public ModifyableTableDescriptor setReadOnly(final boolean readOnly) {
763      return setValue(READONLY_KEY, Boolean.toString(readOnly));
764    }
765
766    /**
767     * The HDFS erasure coding policy for a table. This will be set on the data dir of the table,
768     * and is an alternative to normal replication which takes less space at the cost of locality.
769     * @return the current policy, or null if undefined
770     */
771    @Override
772    public String getErasureCodingPolicy() {
773      return getValue(ERASURE_CODING_POLICY);
774    }
775
776    /**
777     * Sets the HDFS erasure coding policy for the table. This will be propagated to HDFS for the
778     * data dir of the table. Erasure coding is an alternative to normal replication which takes
779     * less space at the cost of locality. The policy must be available and enabled on the hdfs
780     * cluster before being set.
781     * @param policy the policy to set, or null to disable erasure coding
782     * @return the modifyable TD
783     */
784    public ModifyableTableDescriptor setErasureCodingPolicy(String policy) {
785      return setValue(ERASURE_CODING_POLICY_KEY, policy);
786    }
787
788    /**
789     * Check if the compaction enable flag of the table is true. If flag is false then no
790     * minor/major compactions will be done in real.
791     * @return true if table compaction enabled
792     */
793    @Override
794    public boolean isCompactionEnabled() {
795      return getOrDefault(COMPACTION_ENABLED_KEY, Boolean::valueOf, DEFAULT_COMPACTION_ENABLED);
796    }
797
798    /**
799     * Setting the table compaction enable flag.
800     * @param isEnable True if enable compaction.
801     * @return the modifyable TD
802     */
803    public ModifyableTableDescriptor setCompactionEnabled(final boolean isEnable) {
804      return setValue(COMPACTION_ENABLED_KEY, Boolean.toString(isEnable));
805    }
806
807    /**
808     * Check if the split enable flag of the table is true. If flag is false then no split will be
809     * done.
810     * @return true if table region split enabled
811     */
812    @Override
813    public boolean isSplitEnabled() {
814      return getOrDefault(SPLIT_ENABLED_KEY, Boolean::valueOf, DEFAULT_SPLIT_ENABLED);
815    }
816
817    /**
818     * Setting the table region split enable flag.
819     * @param isEnable True if enable region split.
820     * @return the modifyable TD
821     */
822    public ModifyableTableDescriptor setSplitEnabled(final boolean isEnable) {
823      return setValue(SPLIT_ENABLED_KEY, Boolean.toString(isEnable));
824    }
825
826    /**
827     * Check if the region merge enable flag of the table is true. If flag is false then no merge
828     * will be done.
829     * @return true if table region merge enabled
830     */
831    @Override
832    public boolean isMergeEnabled() {
833      return getOrDefault(MERGE_ENABLED_KEY, Boolean::valueOf, DEFAULT_MERGE_ENABLED);
834    }
835
836    /**
837     * Setting the table region merge enable flag.
838     * @param isEnable True if enable region merge.
839     * @return the modifyable TD
840     */
841    public ModifyableTableDescriptor setMergeEnabled(final boolean isEnable) {
842      return setValue(MERGE_ENABLED_KEY, Boolean.toString(isEnable));
843    }
844
845    /**
846     * Check if normalization enable flag of the table is true. If flag is false then no region
847     * normalizer won't attempt to normalize this table.
848     * @return true if region normalization is enabled for this table
849     **/
850    @Override
851    public boolean isNormalizationEnabled() {
852      return getOrDefault(NORMALIZATION_ENABLED_KEY, Boolean::valueOf, false);
853    }
854
855    /**
856     * Check if there is the target region count. If so, the normalize plan will be calculated based
857     * on the target region count.
858     * @return target region count after normalize done
859     */
860    @Override
861    public int getNormalizerTargetRegionCount() {
862      return getOrDefault(NORMALIZER_TARGET_REGION_COUNT_KEY, Integer::valueOf,
863        Integer.valueOf(-1));
864    }
865
866    /**
867     * Check if there is the target region size. If so, the normalize plan will be calculated based
868     * on the target region size.
869     * @return target region size after normalize done
870     */
871    @Override
872    public long getNormalizerTargetRegionSize() {
873      long target_region_size =
874        getOrDefault(NORMALIZER_TARGET_REGION_SIZE_MB_KEY, Long::valueOf, Long.valueOf(-1));
875      return target_region_size == Long.valueOf(-1)
876        ? getOrDefault(NORMALIZER_TARGET_REGION_SIZE_KEY, Long::valueOf, Long.valueOf(-1))
877        : target_region_size;
878    }
879
880    /**
881     * Setting the table normalization enable flag.
882     * @param isEnable True if enable normalization.
883     * @return the modifyable TD
884     */
885    public ModifyableTableDescriptor setNormalizationEnabled(final boolean isEnable) {
886      return setValue(NORMALIZATION_ENABLED_KEY, Boolean.toString(isEnable));
887    }
888
889    /**
890     * Setting the target region count of table normalization .
891     * @param regionCount the target region count.
892     * @return the modifyable TD
893     */
894    public ModifyableTableDescriptor setNormalizerTargetRegionCount(final int regionCount) {
895      return setValue(NORMALIZER_TARGET_REGION_COUNT_KEY, Integer.toString(regionCount));
896    }
897
898    /**
899     * Setting the target region size of table normalization.
900     * @param regionSize the target region size.
901     * @return the modifyable TD
902     */
903    public ModifyableTableDescriptor setNormalizerTargetRegionSize(final long regionSize) {
904      return setValue(NORMALIZER_TARGET_REGION_SIZE_MB_KEY, Long.toString(regionSize));
905    }
906
907    /**
908     * Sets the {@link Durability} setting for the table. This defaults to Durability.USE_DEFAULT.
909     * @param durability enum value
910     * @return the modifyable TD
911     */
912    public ModifyableTableDescriptor setDurability(Durability durability) {
913      return setValue(DURABILITY_KEY, durability.name());
914    }
915
916    /**
917     * Returns the durability setting for the table.
918     * @return durability setting for the table.
919     */
920    @Override
921    public Durability getDurability() {
922      return getOrDefault(DURABILITY_KEY, Durability::valueOf, DEFAULT_DURABLITY);
923    }
924
925    /**
926     * Get the name of the table
927     */
928    @Override
929    public TableName getTableName() {
930      return name;
931    }
932
933    /**
934     * This sets the class associated with the region split policy which determines when a region
935     * split should occur. The class used by default is defined in
936     * org.apache.hadoop.hbase.regionserver.RegionSplitPolicy
937     * @param clazz the class name
938     * @return the modifyable TD
939     */
940    public ModifyableTableDescriptor setRegionSplitPolicyClassName(String clazz) {
941      return setValue(SPLIT_POLICY_KEY, clazz);
942    }
943
944    /**
945     * This gets the class associated with the region split policy which determines when a region
946     * split should occur. The class used by default is defined in
947     * org.apache.hadoop.hbase.regionserver.RegionSplitPolicy
948     * @return the class name of the region split policy for this table. If this returns null, the
949     *         default split policy is used.
950     */
951    @Override
952    public String getRegionSplitPolicyClassName() {
953      return getOrDefault(SPLIT_POLICY_KEY, Function.identity(), null);
954    }
955
956    /**
957     * Returns the maximum size upto which a region can grow to after which a region split is
958     * triggered. The region size is represented by the size of the biggest store file in that
959     * region.
960     * @return max hregion size for table, -1 if not set.
961     * @see #setMaxFileSize(long)
962     */
963    @Override
964    public long getMaxFileSize() {
965      return getOrDefault(MAX_FILESIZE_KEY, Long::valueOf, (long) -1);
966    }
967
968    /**
969     * Sets the maximum size upto which a region can grow to after which a region split is
970     * triggered. The region size is represented by the size of the biggest store file in that
971     * region, i.e. If the biggest store file grows beyond the maxFileSize, then the region split is
972     * triggered. This defaults to a value of 256 MB.
973     * <p>
974     * This is not an absolute value and might vary. Assume that a single row exceeds the
975     * maxFileSize then the storeFileSize will be greater than maxFileSize since a single row cannot
976     * be split across multiple regions
977     * </p>
978     * @param maxFileSize The maximum file size that a store file can grow to before a split is
979     *                    triggered.
980     * @return the modifyable TD
981     */
982    public ModifyableTableDescriptor setMaxFileSize(long maxFileSize) {
983      return setValue(MAX_FILESIZE_KEY, Long.toString(maxFileSize));
984    }
985
986    public ModifyableTableDescriptor setMaxFileSize(String maxFileSize) throws HBaseException {
987      return setMaxFileSize(
988        Long.parseLong(PrettyPrinter.valueOf(maxFileSize, PrettyPrinter.Unit.BYTE)));
989    }
990
991    /**
992     * Returns the size of the memstore after which a flush to filesystem is triggered.
993     * @return memory cache flush size for each hregion, -1 if not set.
994     * @see #setMemStoreFlushSize(long)
995     */
996    @Override
997    public long getMemStoreFlushSize() {
998      return getOrDefault(MEMSTORE_FLUSHSIZE_KEY, Long::valueOf, (long) -1);
999    }
1000
1001    /**
1002     * Represents the maximum size of the memstore after which the contents of the memstore are
1003     * flushed to the filesystem. This defaults to a size of 64 MB.
1004     * @param memstoreFlushSize memory cache flush size for each hregion
1005     * @return the modifyable TD
1006     */
1007    public ModifyableTableDescriptor setMemStoreFlushSize(long memstoreFlushSize) {
1008      return setValue(MEMSTORE_FLUSHSIZE_KEY, Long.toString(memstoreFlushSize));
1009    }
1010
1011    public ModifyableTableDescriptor setMemStoreFlushSize(String memStoreFlushSize)
1012      throws HBaseException {
1013      return setMemStoreFlushSize(
1014        Long.parseLong(PrettyPrinter.valueOf(memStoreFlushSize, PrettyPrinter.Unit.BYTE)));
1015    }
1016
1017    /**
1018     * This sets the class associated with the flush policy which determines determines the stores
1019     * need to be flushed when flushing a region. The class used by default is defined in
1020     * org.apache.hadoop.hbase.regionserver.FlushPolicy.
1021     * @param clazz the class name
1022     * @return the modifyable TD
1023     */
1024    public ModifyableTableDescriptor setFlushPolicyClassName(String clazz) {
1025      return setValue(FLUSH_POLICY_KEY, clazz);
1026    }
1027
1028    /**
1029     * This gets the class associated with the flush policy which determines the stores need to be
1030     * flushed when flushing a region. The class used by default is defined in
1031     * org.apache.hadoop.hbase.regionserver.FlushPolicy.
1032     * @return the class name of the flush policy for this table. If this returns null, the default
1033     *         flush policy is used.
1034     */
1035    @Override
1036    public String getFlushPolicyClassName() {
1037      return getOrDefault(FLUSH_POLICY_KEY, Function.identity(), null);
1038    }
1039
1040    /**
1041     * Adds a column family. For the updating purpose please use
1042     * {@link #modifyColumnFamily(ColumnFamilyDescriptor)} instead.
1043     * @param family to add.
1044     * @return the modifyable TD
1045     */
1046    public ModifyableTableDescriptor setColumnFamily(final ColumnFamilyDescriptor family) {
1047      if (family.getName() == null || family.getName().length <= 0) {
1048        throw new IllegalArgumentException("Family name cannot be null or empty");
1049      }
1050      int flength = family.getName() == null ? 0 : family.getName().length;
1051      if (flength > Byte.MAX_VALUE) {
1052        throw new IllegalArgumentException(
1053          "The length of family name is bigger than " + Byte.MAX_VALUE);
1054      }
1055      if (hasColumnFamily(family.getName())) {
1056        throw new IllegalArgumentException(
1057          "Family '" + family.getNameAsString() + "' already exists so cannot be added");
1058      }
1059      return putColumnFamily(family);
1060    }
1061
1062    /**
1063     * Modifies the existing column family.
1064     * @param family to update
1065     * @return this (for chained invocation)
1066     */
1067    public ModifyableTableDescriptor modifyColumnFamily(final ColumnFamilyDescriptor family) {
1068      if (family.getName() == null || family.getName().length <= 0) {
1069        throw new IllegalArgumentException("Family name cannot be null or empty");
1070      }
1071      if (!hasColumnFamily(family.getName())) {
1072        throw new IllegalArgumentException(
1073          "Column family '" + family.getNameAsString() + "' does not exist");
1074      }
1075      return putColumnFamily(family);
1076    }
1077
1078    private ModifyableTableDescriptor putColumnFamily(ColumnFamilyDescriptor family) {
1079      families.put(family.getName(), family);
1080      return this;
1081    }
1082
1083    /**
1084     * Checks to see if this table contains the given column family
1085     * @param familyName Family name or column name.
1086     * @return true if the table contains the specified family name
1087     */
1088    @Override
1089    public boolean hasColumnFamily(final byte[] familyName) {
1090      return families.containsKey(familyName);
1091    }
1092
1093    /** Returns Name of this table and then a map of all of the column family descriptors. */
1094    @Override
1095    public String toString() {
1096      StringBuilder s = new StringBuilder();
1097      s.append('\'').append(Bytes.toString(name.getName())).append('\'');
1098      s.append(getValues(true));
1099      families.values().forEach(f -> s.append(", ").append(f));
1100      return s.toString();
1101    }
1102
1103    /**
1104     * @return Name of this table and then a map of all of the column family descriptors (with only
1105     *         the non-default column family attributes)
1106     */
1107    @Override
1108    public String toStringCustomizedValues() {
1109      StringBuilder s = new StringBuilder();
1110      s.append('\'').append(Bytes.toString(name.getName())).append('\'');
1111      s.append(getValues(false));
1112      families.values().forEach(hcd -> s.append(", ").append(hcd.toStringCustomizedValues()));
1113      return s.toString();
1114    }
1115
1116    /** Returns map of all table attributes formatted into string. */
1117    public String toStringTableAttributes() {
1118      return getValues(true).toString();
1119    }
1120
1121    private StringBuilder getValues(boolean printDefaults) {
1122      StringBuilder s = new StringBuilder();
1123
1124      // step 1: set partitioning and pruning
1125      Set<Bytes> reservedKeys = new TreeSet<>();
1126      Set<Bytes> userKeys = new TreeSet<>();
1127      for (Map.Entry<Bytes, Bytes> entry : values.entrySet()) {
1128        if (entry.getKey() == null || entry.getKey().get() == null) {
1129          continue;
1130        }
1131        String key = Bytes.toString(entry.getKey().get());
1132        // in this section, print out reserved keywords + coprocessor info
1133        if (!RESERVED_KEYWORDS.contains(entry.getKey()) && !key.startsWith("coprocessor$")) {
1134          userKeys.add(entry.getKey());
1135          continue;
1136        }
1137        // only print out IS_META if true
1138        String value = Bytes.toString(entry.getValue().get());
1139        if (key.equalsIgnoreCase(IS_META)) {
1140          if (Boolean.valueOf(value) == false) {
1141            continue;
1142          }
1143        }
1144        // see if a reserved key is a default value. may not want to print it out
1145        if (
1146          printDefaults || !DEFAULT_VALUES.containsKey(key)
1147            || !DEFAULT_VALUES.get(key).equalsIgnoreCase(value)
1148        ) {
1149          reservedKeys.add(entry.getKey());
1150        }
1151      }
1152
1153      // early exit optimization
1154      boolean hasAttributes = !reservedKeys.isEmpty() || !userKeys.isEmpty();
1155      if (!hasAttributes) {
1156        return s;
1157      }
1158
1159      s.append(", {");
1160      // step 2: printing attributes
1161      if (hasAttributes) {
1162        s.append("TABLE_ATTRIBUTES => {");
1163
1164        // print all reserved keys first
1165        boolean printCommaForAttr = false;
1166        for (Bytes k : reservedKeys) {
1167          String key = Bytes.toString(k.get());
1168          String value = Bytes.toStringBinary(values.get(k).get());
1169          if (printCommaForAttr) {
1170            s.append(", ");
1171          }
1172          printCommaForAttr = true;
1173          s.append(key);
1174          s.append(" => ");
1175          s.append('\'').append(PrettyPrinter.format(value, getUnit(key))).append('\'');
1176        }
1177
1178        if (!userKeys.isEmpty()) {
1179          // print all non-reserved as a separate subset
1180          if (printCommaForAttr) {
1181            s.append(", ");
1182          }
1183          s.append(HConstants.METADATA).append(" => ");
1184          s.append("{");
1185          boolean printCommaForCfg = false;
1186          for (Bytes k : userKeys) {
1187            String key = Bytes.toString(k.get());
1188            String value = Bytes.toStringBinary(values.get(k).get());
1189            if (printCommaForCfg) {
1190              s.append(", ");
1191            }
1192            printCommaForCfg = true;
1193            s.append('\'').append(key).append('\'');
1194            s.append(" => ");
1195            s.append('\'').append(PrettyPrinter.format(value, getUnit(key))).append('\'');
1196          }
1197          s.append("}");
1198        }
1199
1200        s.append("}");
1201      }
1202
1203      s.append("}"); // end METHOD
1204      return s;
1205    }
1206
1207    /**
1208     * Compare the contents of the descriptor with another one passed as a parameter. Checks if the
1209     * obj passed is an instance of ModifyableTableDescriptor, if yes then the contents of the
1210     * descriptors are compared.
1211     * @param obj The object to compare
1212     * @return true if the contents of the the two descriptors exactly match
1213     * @see java.lang.Object#equals(java.lang.Object)
1214     */
1215    @Override
1216    public boolean equals(Object obj) {
1217      if (this == obj) {
1218        return true;
1219      }
1220      if (obj instanceof ModifyableTableDescriptor) {
1221        return TableDescriptor.COMPARATOR.compare(this, (ModifyableTableDescriptor) obj) == 0;
1222      }
1223      return false;
1224    }
1225
1226    /** Returns hash code */
1227    @Override
1228    public int hashCode() {
1229      int result = this.name.hashCode();
1230      if (this.families.size() > 0) {
1231        for (ColumnFamilyDescriptor e : this.families.values()) {
1232          result ^= e.hashCode();
1233        }
1234      }
1235      result ^= values.hashCode();
1236      return result;
1237    }
1238
1239    // Comparable
1240    /**
1241     * Compares the descriptor with another descriptor which is passed as a parameter. This compares
1242     * the content of the two descriptors and not the reference.
1243     * @param other The MTD to compare
1244     * @return 0 if the contents of the descriptors are exactly matching, 1 if there is a mismatch
1245     *         in the contents
1246     */
1247    @Override
1248    public int compareTo(final ModifyableTableDescriptor other) {
1249      return TableDescriptor.COMPARATOR.compare(this, other);
1250    }
1251
1252    @Override
1253    public ColumnFamilyDescriptor[] getColumnFamilies() {
1254      return families.values().toArray(new ColumnFamilyDescriptor[families.size()]);
1255    }
1256
1257    /**
1258     * Returns the configured replicas per region
1259     */
1260    @Override
1261    public int getRegionReplication() {
1262      return getOrDefault(REGION_REPLICATION_KEY, Integer::valueOf, DEFAULT_REGION_REPLICATION);
1263    }
1264
1265    /**
1266     * Sets the number of replicas per region.
1267     * @param regionReplication the replication factor per region
1268     * @return the modifyable TD
1269     */
1270    public ModifyableTableDescriptor setRegionReplication(int regionReplication) {
1271      return setValue(REGION_REPLICATION_KEY, Integer.toString(regionReplication));
1272    }
1273
1274    /** Returns true if the read-replicas memstore replication is enabled. */
1275    @Override
1276    public boolean hasRegionMemStoreReplication() {
1277      return getOrDefault(REGION_MEMSTORE_REPLICATION_KEY, Boolean::valueOf,
1278        DEFAULT_REGION_MEMSTORE_REPLICATION);
1279    }
1280
1281    /**
1282     * Enable or Disable the memstore replication from the primary region to the replicas. The
1283     * replication will be used only for meta operations (e.g. flush, compaction, ...)
1284     * @param memstoreReplication true if the new data written to the primary region should be
1285     *                            replicated. false if the secondaries can tollerate to have new
1286     *                            data only when the primary flushes the memstore.
1287     * @return the modifyable TD
1288     */
1289    public ModifyableTableDescriptor setRegionMemStoreReplication(boolean memstoreReplication) {
1290      return setValue(REGION_MEMSTORE_REPLICATION_KEY, Boolean.toString(memstoreReplication));
1291    }
1292
1293    public ModifyableTableDescriptor setPriority(int priority) {
1294      return setValue(PRIORITY_KEY, Integer.toString(priority));
1295    }
1296
1297    @Override
1298    public int getPriority() {
1299      return getOrDefault(PRIORITY_KEY, Integer::valueOf, DEFAULT_PRIORITY);
1300    }
1301
1302    /**
1303     * Returns all the column family names of the current table. The map of TableDescriptor contains
1304     * mapping of family name to ColumnFamilyDescriptor. This returns all the keys of the family map
1305     * which represents the column family names of the table.
1306     * @return Immutable sorted set of the keys of the families.
1307     */
1308    @Override
1309    public Set<byte[]> getColumnFamilyNames() {
1310      return Collections.unmodifiableSet(this.families.keySet());
1311    }
1312
1313    /**
1314     * Returns the ColumnFamilyDescriptor for a specific column family with name as specified by the
1315     * parameter column.
1316     * @param column Column family name
1317     * @return Column descriptor for the passed family name or the family on passed in column.
1318     */
1319    @Override
1320    public ColumnFamilyDescriptor getColumnFamily(final byte[] column) {
1321      return this.families.get(column);
1322    }
1323
1324    /**
1325     * Removes the ColumnFamilyDescriptor with name specified by the parameter column from the table
1326     * descriptor
1327     * @param column Name of the column family to be removed.
1328     * @return Column descriptor for the passed family name or the family on passed in column.
1329     */
1330    public ColumnFamilyDescriptor removeColumnFamily(final byte[] column) {
1331      return this.families.remove(column);
1332    }
1333
1334    /**
1335     * Add a table coprocessor to this table. The coprocessor type must be
1336     * org.apache.hadoop.hbase.coprocessor.RegionObserver or Endpoint. It won't check if the class
1337     * can be loaded or not. Whether a coprocessor is loadable or not will be determined when a
1338     * region is opened.
1339     * @param className Full class name.
1340     * @return the modifyable TD
1341     */
1342    public ModifyableTableDescriptor setCoprocessor(String className) throws IOException {
1343      return setCoprocessor(CoprocessorDescriptorBuilder.newBuilder(className)
1344        .setPriority(Coprocessor.PRIORITY_USER).build());
1345    }
1346
1347    /**
1348     * Add a table coprocessor to this table. The coprocessor type must be
1349     * org.apache.hadoop.hbase.coprocessor.RegionObserver or Endpoint. It won't check if the class
1350     * can be loaded or not. Whether a coprocessor is loadable or not will be determined when a
1351     * region is opened.
1352     * @throws IOException any illegal parameter key/value
1353     * @return the modifyable TD
1354     */
1355    public ModifyableTableDescriptor setCoprocessor(CoprocessorDescriptor cp) throws IOException {
1356      checkHasCoprocessor(cp.getClassName());
1357      if (cp.getPriority() < 0) {
1358        throw new IOException(
1359          "Priority must be bigger than or equal with zero, current:" + cp.getPriority());
1360      }
1361      // Validate parameter kvs and then add key/values to kvString.
1362      StringBuilder kvString = new StringBuilder();
1363      for (Map.Entry<String, String> e : cp.getProperties().entrySet()) {
1364        if (!e.getKey().matches(CP_HTD_ATTR_VALUE_PARAM_KEY_PATTERN)) {
1365          throw new IOException("Illegal parameter key = " + e.getKey());
1366        }
1367        if (!e.getValue().matches(CP_HTD_ATTR_VALUE_PARAM_VALUE_PATTERN)) {
1368          throw new IOException("Illegal parameter (" + e.getKey() + ") value = " + e.getValue());
1369        }
1370        if (kvString.length() != 0) {
1371          kvString.append(',');
1372        }
1373        kvString.append(e.getKey());
1374        kvString.append('=');
1375        kvString.append(e.getValue());
1376      }
1377
1378      String value = cp.getJarPath().orElse("") + "|" + cp.getClassName() + "|"
1379        + Integer.toString(cp.getPriority()) + "|" + kvString.toString();
1380      return setCoprocessorToMap(value);
1381    }
1382
1383    /**
1384     * Add a table coprocessor to this table. The coprocessor type must be
1385     * org.apache.hadoop.hbase.coprocessor.RegionObserver or Endpoint. It won't check if the class
1386     * can be loaded or not. Whether a coprocessor is loadable or not will be determined when a
1387     * region is opened.
1388     * @param specStr The Coprocessor specification all in in one String
1389     * @return the modifyable TD
1390     * @deprecated used by HTableDescriptor and admin.rb. As of release 2.0.0, this will be removed
1391     *             in HBase 3.0.0.
1392     */
1393    @Deprecated
1394    public ModifyableTableDescriptor setCoprocessorWithSpec(final String specStr)
1395      throws IOException {
1396      CoprocessorDescriptor cpDesc =
1397        toCoprocessorDescriptor(specStr).orElseThrow(() -> new IllegalArgumentException(
1398          "Format does not match " + CP_HTD_ATTR_VALUE_PATTERN + ": " + specStr));
1399      checkHasCoprocessor(cpDesc.getClassName());
1400      return setCoprocessorToMap(specStr);
1401    }
1402
1403    private void checkHasCoprocessor(final String className) throws IOException {
1404      if (hasCoprocessor(className)) {
1405        throw new IOException("Coprocessor " + className + " already exists.");
1406      }
1407    }
1408
1409    /**
1410     * Add coprocessor to values Map
1411     * @param specStr The Coprocessor specification all in in one String
1412     * @return Returns <code>this</code>
1413     */
1414    private ModifyableTableDescriptor setCoprocessorToMap(final String specStr) {
1415      if (specStr == null) {
1416        return this;
1417      }
1418      // generate a coprocessor key
1419      int maxCoprocessorNumber = 0;
1420      Matcher keyMatcher;
1421      for (Map.Entry<Bytes, Bytes> e : this.values.entrySet()) {
1422        keyMatcher = CP_HTD_ATTR_KEY_PATTERN.matcher(Bytes.toString(e.getKey().get()));
1423        if (!keyMatcher.matches()) {
1424          continue;
1425        }
1426        maxCoprocessorNumber =
1427          Math.max(Integer.parseInt(keyMatcher.group(1)), maxCoprocessorNumber);
1428      }
1429      maxCoprocessorNumber++;
1430      String key = "coprocessor$" + Integer.toString(maxCoprocessorNumber);
1431      return setValue(new Bytes(Bytes.toBytes(key)), new Bytes(Bytes.toBytes(specStr)));
1432    }
1433
1434    /**
1435     * Check if the table has an attached co-processor represented by the name className
1436     * @param classNameToMatch - Class name of the co-processor
1437     * @return true of the table has a co-processor className
1438     */
1439    @Override
1440    public boolean hasCoprocessor(String classNameToMatch) {
1441      return getCoprocessorDescriptors().stream()
1442        .anyMatch(cp -> cp.getClassName().equals(classNameToMatch));
1443    }
1444
1445    /**
1446     * Return the list of attached co-processor represented by their name className
1447     * @return The list of co-processors classNames
1448     */
1449    @Override
1450    public List<CoprocessorDescriptor> getCoprocessorDescriptors() {
1451      List<CoprocessorDescriptor> result = new ArrayList<>();
1452      for (Map.Entry<Bytes, Bytes> e : getValues().entrySet()) {
1453        String key = Bytes.toString(e.getKey().get()).trim();
1454        if (CP_HTD_ATTR_KEY_PATTERN.matcher(key).matches()) {
1455          toCoprocessorDescriptor(Bytes.toString(e.getValue().get()).trim()).ifPresent(result::add);
1456        }
1457      }
1458      return result;
1459    }
1460
1461    /**
1462     * Remove a coprocessor from those set on the table
1463     * @param className Class name of the co-processor
1464     */
1465    public void removeCoprocessor(String className) {
1466      Bytes match = null;
1467      Matcher keyMatcher;
1468      Matcher valueMatcher;
1469      for (Map.Entry<Bytes, Bytes> e : this.values.entrySet()) {
1470        keyMatcher = CP_HTD_ATTR_KEY_PATTERN.matcher(Bytes.toString(e.getKey().get()));
1471        if (!keyMatcher.matches()) {
1472          continue;
1473        }
1474        valueMatcher = CP_HTD_ATTR_VALUE_PATTERN.matcher(Bytes.toString(e.getValue().get()));
1475        if (!valueMatcher.matches()) {
1476          continue;
1477        }
1478        // get className and compare
1479        String clazz = valueMatcher.group(2).trim(); // classname is the 2nd field
1480        // remove the CP if it is present
1481        if (clazz.equals(className.trim())) {
1482          match = e.getKey();
1483          break;
1484        }
1485      }
1486      // if we found a match, remove it
1487      if (match != null) {
1488        ModifyableTableDescriptor.this.removeValue(match);
1489      } else {
1490        throw new IllegalArgumentException(String.format(
1491          "coprocessor with class name %s was not found in the table attribute", className));
1492      }
1493    }
1494
1495    /** Returns the bytes in pb format */
1496    private byte[] toByteArray() {
1497      return ProtobufUtil.prependPBMagic(ProtobufUtil.toTableSchema(this).toByteArray());
1498    }
1499
1500    /**
1501     * @param bytes A pb serialized {@link ModifyableTableDescriptor} instance with pb magic prefix
1502     * @return An instance of {@link ModifyableTableDescriptor} made from <code>bytes</code>
1503     * @see #toByteArray()
1504     */
1505    private static TableDescriptor parseFrom(final byte[] bytes) throws DeserializationException {
1506      if (!ProtobufUtil.isPBMagicPrefix(bytes)) {
1507        throw new DeserializationException("Expected PB encoded ModifyableTableDescriptor");
1508      }
1509      int pblen = ProtobufUtil.lengthOfPBMagic();
1510      HBaseProtos.TableSchema.Builder builder = HBaseProtos.TableSchema.newBuilder();
1511      try {
1512        ProtobufUtil.mergeFrom(builder, bytes, pblen, bytes.length - pblen);
1513        return ProtobufUtil.toTableDescriptor(builder.build());
1514      } catch (IOException e) {
1515        throw new DeserializationException(e);
1516      }
1517    }
1518
1519    @Override
1520    public int getColumnFamilyCount() {
1521      return families.size();
1522    }
1523
1524    @Override
1525    public Optional<String> getRegionServerGroup() {
1526      Bytes value = values.get(RSGROUP_KEY);
1527      if (value != null) {
1528        return Optional.of(Bytes.toString(value.get(), value.getOffset(), value.getLength()));
1529      } else {
1530        return Optional.empty();
1531      }
1532    }
1533  }
1534
1535  /**
1536   * This method is mostly intended for internal use. However, it it also relied on by hbase-shell
1537   * for backwards compatibility.
1538   */
1539  private static Optional<CoprocessorDescriptor> toCoprocessorDescriptor(String spec) {
1540    Matcher matcher = CP_HTD_ATTR_VALUE_PATTERN.matcher(spec);
1541    if (matcher.matches()) {
1542      // jar file path can be empty if the cp class can be loaded
1543      // from class loader.
1544      String path = matcher.group(1).trim().isEmpty() ? null : matcher.group(1).trim();
1545      String className = matcher.group(2).trim();
1546      if (className.isEmpty()) {
1547        return Optional.empty();
1548      }
1549      String priorityStr = matcher.group(3).trim();
1550      int priority =
1551        priorityStr.isEmpty() ? Coprocessor.PRIORITY_USER : Integer.parseInt(priorityStr);
1552      String cfgSpec = null;
1553      try {
1554        cfgSpec = matcher.group(4);
1555      } catch (IndexOutOfBoundsException ex) {
1556        // ignore
1557      }
1558      Map<String, String> ourConf = new TreeMap<>();
1559      if (cfgSpec != null && !cfgSpec.trim().equals("|")) {
1560        cfgSpec = cfgSpec.substring(cfgSpec.indexOf('|') + 1);
1561        Matcher m = CP_HTD_ATTR_VALUE_PARAM_PATTERN.matcher(cfgSpec);
1562        while (m.find()) {
1563          ourConf.put(m.group(1), m.group(2));
1564        }
1565      }
1566      return Optional.of(CoprocessorDescriptorBuilder.newBuilder(className).setJarPath(path)
1567        .setPriority(priority).setProperties(ourConf).build());
1568    }
1569    return Optional.empty();
1570  }
1571}