001/**
002 *
003 * Licensed to the Apache Software Foundation (ASF) under one
004 * or more contributor license agreements.  See the NOTICE file
005 * distributed with this work for additional information
006 * regarding copyright ownership.  The ASF licenses this file
007 * to you under the Apache License, Version 2.0 (the
008 * "License"); you may not use this file except in compliance
009 * with the License.  You may obtain a copy of the License at
010 *
011 *     http://www.apache.org/licenses/LICENSE-2.0
012 *
013 * Unless required by applicable law or agreed to in writing, software
014 * distributed under the License is distributed on an "AS IS" BASIS,
015 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
016 * See the License for the specific language governing permissions and
017 * limitations under the License.
018 */
019package org.apache.hadoop.hbase;
020
021import java.io.IOException;
022import java.util.Collection;
023import java.util.Collections;
024import java.util.List;
025import java.util.Map;
026import java.util.Optional;
027import java.util.Set;
028import java.util.stream.Collectors;
029import java.util.stream.Stream;
030import org.apache.hadoop.fs.Path;
031import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
032import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor;
033import org.apache.hadoop.hbase.client.CoprocessorDescriptor;
034import org.apache.hadoop.hbase.client.CoprocessorDescriptorBuilder;
035import org.apache.hadoop.hbase.client.Durability;
036import org.apache.hadoop.hbase.client.TableDescriptor;
037import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
038import org.apache.hadoop.hbase.client.TableDescriptorBuilder.ModifyableTableDescriptor;
039import org.apache.hadoop.hbase.exceptions.DeserializationException;
040import org.apache.hadoop.hbase.security.User;
041import org.apache.hadoop.hbase.util.Bytes;
042import org.apache.yetus.audience.InterfaceAudience;
043
044/**
045 * HTableDescriptor contains the details about an HBase table  such as the descriptors of
046 * all the column families, is the table a catalog table, <code> hbase:meta </code>,
047 * if the table is read only, the maximum size of the memstore,
048 * when the region split should occur, coprocessors associated with it etc...
049 * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0.
050 *             Use {@link TableDescriptorBuilder} to build {@link HTableDescriptor}.
051 */
052@Deprecated
053@InterfaceAudience.Public
054public class HTableDescriptor implements TableDescriptor, Comparable<HTableDescriptor> {
055  public static final String SPLIT_POLICY = TableDescriptorBuilder.SPLIT_POLICY;
056  public static final String MAX_FILESIZE = TableDescriptorBuilder.MAX_FILESIZE;
057  public static final String OWNER = TableDescriptorBuilder.OWNER;
058  public static final Bytes OWNER_KEY = TableDescriptorBuilder.OWNER_KEY;
059  public static final String READONLY = TableDescriptorBuilder.READONLY;
060  public static final String COMPACTION_ENABLED = TableDescriptorBuilder.COMPACTION_ENABLED;
061  public static final String SPLIT_ENABLED = TableDescriptorBuilder.SPLIT_ENABLED;
062  public static final String MERGE_ENABLED = TableDescriptorBuilder.MERGE_ENABLED;
063  public static final String MEMSTORE_FLUSHSIZE = TableDescriptorBuilder.MEMSTORE_FLUSHSIZE;
064  public static final String FLUSH_POLICY = TableDescriptorBuilder.FLUSH_POLICY;
065  public static final String IS_ROOT = "IS_ROOT";
066  public static final String IS_META = TableDescriptorBuilder.IS_META;
067  public static final String DURABILITY = TableDescriptorBuilder.DURABILITY;
068  public static final String REGION_REPLICATION = TableDescriptorBuilder.REGION_REPLICATION;
069  public static final String REGION_MEMSTORE_REPLICATION = TableDescriptorBuilder.REGION_MEMSTORE_REPLICATION;
070  public static final String NORMALIZATION_ENABLED = TableDescriptorBuilder.NORMALIZATION_ENABLED;
071  public static final String NORMALIZER_TARGET_REGION_COUNT =
072      TableDescriptorBuilder.NORMALIZER_TARGET_REGION_COUNT;
073  public static final String NORMALIZER_TARGET_REGION_SIZE =
074      TableDescriptorBuilder.NORMALIZER_TARGET_REGION_SIZE;
075  public static final String PRIORITY = TableDescriptorBuilder.PRIORITY;
076  public static final boolean DEFAULT_READONLY = TableDescriptorBuilder.DEFAULT_READONLY;
077  public static final boolean DEFAULT_COMPACTION_ENABLED = TableDescriptorBuilder.DEFAULT_COMPACTION_ENABLED;
078  public static final boolean DEFAULT_NORMALIZATION_ENABLED = TableDescriptorBuilder.DEFAULT_NORMALIZATION_ENABLED;
079  public static final long DEFAULT_MEMSTORE_FLUSH_SIZE = TableDescriptorBuilder.DEFAULT_MEMSTORE_FLUSH_SIZE;
080  public static final int DEFAULT_REGION_REPLICATION = TableDescriptorBuilder.DEFAULT_REGION_REPLICATION;
081  public static final boolean DEFAULT_REGION_MEMSTORE_REPLICATION = TableDescriptorBuilder.DEFAULT_REGION_MEMSTORE_REPLICATION;
082  protected final ModifyableTableDescriptor delegatee;
083
084  /**
085   * Construct a table descriptor specifying a TableName object
086   * @param name Table name.
087   * @see <a href="https://issues.apache.org/jira/browse/HBASE-174">HADOOP-1581 HBASE: (HBASE-174) Un-openable tablename bug</a>
088   */
089  public HTableDescriptor(final TableName name) {
090    this(new ModifyableTableDescriptor(name));
091  }
092
093  /**
094   * Construct a table descriptor by cloning the descriptor passed as a parameter.
095   * <p>
096   * Makes a deep copy of the supplied descriptor.
097   * Can make a modifiable descriptor from an ImmutableHTableDescriptor.
098   * @param desc The descriptor.
099   */
100  public HTableDescriptor(final HTableDescriptor desc) {
101    this(desc, true);
102  }
103
104  protected HTableDescriptor(final HTableDescriptor desc, boolean deepClone) {
105    this(deepClone ? new ModifyableTableDescriptor(desc.getTableName(), desc)
106      : desc.delegatee);
107  }
108
109  public HTableDescriptor(final TableDescriptor desc) {
110    this(new ModifyableTableDescriptor(desc.getTableName(), desc));
111  }
112
113  /**
114   * Construct a table descriptor by cloning the descriptor passed as a parameter
115   * but using a different table name.
116   * <p>
117   * Makes a deep copy of the supplied descriptor.
118   * Can make a modifiable descriptor from an ImmutableHTableDescriptor.
119   * @param name Table name.
120   * @param desc The descriptor.
121   */
122  public HTableDescriptor(final TableName name, final HTableDescriptor desc) {
123    this(new ModifyableTableDescriptor(name, desc));
124  }
125
126  protected HTableDescriptor(ModifyableTableDescriptor delegatee) {
127    this.delegatee = delegatee;
128  }
129
130  /**
131   * This is vestigial API. It will be removed in 3.0.
132   *
133   * @return always return the false
134   */
135  public boolean isRootRegion() {
136    return false;
137  }
138
139  /**
140   * Checks if this table is <code> hbase:meta </code>
141   * region.
142   *
143   * @return true if this table is <code> hbase:meta </code>
144   * region
145   */
146  @Override
147  public boolean isMetaRegion() {
148    return delegatee.isMetaRegion();
149  }
150
151  /**
152   * Checks if the table is a <code>hbase:meta</code> table
153   *
154   * @return true if table is <code> hbase:meta </code> region.
155   */
156  @Override
157  public boolean isMetaTable() {
158    return delegatee.isMetaTable();
159  }
160
161  /**
162   * @return Getter for fetching an unmodifiable map.
163   */
164  @Override
165  public Map<Bytes, Bytes> getValues() {
166    return delegatee.getValues();
167  }
168
169  /**
170   * Setter for storing metadata as a (key, value) pair in map
171   *
172   * @param key The key.
173   * @param value The value. If null, removes the setting.
174   */
175  public HTableDescriptor setValue(byte[] key, byte[] value) {
176    getDelegateeForModification().setValue(key, value);
177    return this;
178  }
179
180  /*
181   * Setter for storing metadata as a (key, value) pair in map
182   *
183   * @param key The key.
184   * @param value The value. If null, removes the setting.
185   */
186  public HTableDescriptor setValue(final Bytes key, final Bytes value) {
187    getDelegateeForModification().setValue(key, value);
188    return this;
189  }
190
191  /**
192   * Setter for storing metadata as a (key, value) pair in map
193   *
194   * @param key The key.
195   * @param value The value. If null, removes the setting.
196   */
197  public HTableDescriptor setValue(String key, String value) {
198    getDelegateeForModification().setValue(key, value);
199    return this;
200  }
201
202  /**
203   * Remove metadata represented by the key from the map
204   *
205   * @param key Key whose key and value we're to remove from HTableDescriptor
206   * parameters.
207   */
208  public void remove(final String key) {
209    getDelegateeForModification().removeValue(Bytes.toBytes(key));
210  }
211
212  /**
213   * Remove metadata represented by the key from the map
214   *
215   * @param key Key whose key and value we're to remove from HTableDescriptor
216   * parameters.
217   */
218  public void remove(Bytes key) {
219    getDelegateeForModification().removeValue(key);
220  }
221
222  /**
223   * Remove metadata represented by the key from the map
224   *
225   * @param key Key whose key and value we're to remove from HTableDescriptor
226   * parameters.
227   */
228  public void remove(final byte [] key) {
229    getDelegateeForModification().removeValue(key);
230  }
231
232  /**
233   * Check if the readOnly flag of the table is set. If the readOnly flag is
234   * set then the contents of the table can only be read from but not modified.
235   *
236   * @return true if all columns in the table should be read only
237   */
238  @Override
239  public boolean isReadOnly() {
240    return delegatee.isReadOnly();
241  }
242
243  /**
244   * Setting the table as read only sets all the columns in the table as read
245   * only. By default all tables are modifiable, but if the readOnly flag is
246   * set to true then the contents of the table can only be read but not modified.
247   *
248   * @param readOnly True if all of the columns in the table should be read
249   * only.
250   */
251  public HTableDescriptor setReadOnly(final boolean readOnly) {
252    getDelegateeForModification().setReadOnly(readOnly);
253    return this;
254  }
255
256  /**
257   * Check if the compaction enable flag of the table is true. If flag is
258   * false then no minor/major compactions will be done in real.
259   *
260   * @return true if table compaction enabled
261   */
262  @Override
263  public boolean isCompactionEnabled() {
264    return delegatee.isCompactionEnabled();
265  }
266
267  /**
268   * Setting the table compaction enable flag.
269   *
270   * @param isEnable True if enable compaction.
271   */
272  public HTableDescriptor setCompactionEnabled(final boolean isEnable) {
273    getDelegateeForModification().setCompactionEnabled(isEnable);
274    return this;
275  }
276
277  /**
278   * Check if the region split enable flag of the table is true. If flag is
279   * false then no split will be done.
280   *
281   * @return true if table region split enabled
282   */
283  @Override
284  public boolean isSplitEnabled() {
285    return delegatee.isSplitEnabled();
286  }
287
288  /**
289   * Setting the table region split enable flag.
290   *
291   * @param isEnable True if enable split.
292   */
293  public HTableDescriptor setSplitEnabled(final boolean isEnable) {
294    getDelegateeForModification().setSplitEnabled(isEnable);
295    return this;
296  }
297
298
299  /**
300   * Check if the region merge enable flag of the table is true. If flag is
301   * false then no merge will be done.
302   *
303   * @return true if table region merge enabled
304   */
305  @Override
306  public boolean isMergeEnabled() {
307    return delegatee.isMergeEnabled();
308  }
309
310  /**
311   * Setting the table region merge enable flag.
312   *
313   * @param isEnable True if enable merge.
314   */
315  public HTableDescriptor setMergeEnabled(final boolean isEnable) {
316    getDelegateeForModification().setMergeEnabled(isEnable);
317    return this;
318  }
319
320  /**
321   * Check if normalization enable flag of the table is true. If flag is
322   * false then no region normalizer won't attempt to normalize this table.
323   *
324   * @return true if region normalization is enabled for this table
325   */
326  @Override
327  public boolean isNormalizationEnabled() {
328    return delegatee.isNormalizationEnabled();
329  }
330
331  /**
332   * Setting the table normalization enable flag.
333   *
334   * @param isEnable True if enable normalization.
335   */
336  public HTableDescriptor setNormalizationEnabled(final boolean isEnable) {
337    getDelegateeForModification().setNormalizationEnabled(isEnable);
338    return this;
339  }
340
341  @Override
342  public int getNormalizerTargetRegionCount() {
343    return getDelegateeForModification().getNormalizerTargetRegionCount();
344  }
345
346  public HTableDescriptor setNormalizerTargetRegionCount(final int regionCount) {
347    getDelegateeForModification().setNormalizerTargetRegionCount(regionCount);
348    return this;
349  }
350
351  @Override
352  public long getNormalizerTargetRegionSize() {
353    return getDelegateeForModification().getNormalizerTargetRegionSize();
354  }
355
356  public HTableDescriptor setNormalizerTargetRegionSize(final long regionSize) {
357    getDelegateeForModification().setNormalizerTargetRegionSize(regionSize);
358    return this;
359  }
360
361  /**
362   * Sets the {@link Durability} setting for the table. This defaults to Durability.USE_DEFAULT.
363   * @param durability enum value
364   */
365  public HTableDescriptor setDurability(Durability durability) {
366    getDelegateeForModification().setDurability(durability);
367    return this;
368  }
369
370  /**
371   * Returns the durability setting for the table.
372   * @return durability setting for the table.
373   */
374  @Override
375  public Durability getDurability() {
376    return delegatee.getDurability();
377  }
378
379  /**
380   * Get the name of the table
381   *
382   * @return TableName
383   */
384  @Override
385  public TableName getTableName() {
386    return delegatee.getTableName();
387  }
388
389  /**
390   * Get the name of the table as a String
391   *
392   * @return name of table as a String
393   */
394  public String getNameAsString() {
395    return delegatee.getTableName().getNameAsString();
396  }
397
398  /**
399   * This sets the class associated with the region split policy which
400   * determines when a region split should occur.  The class used by
401   * default is defined in org.apache.hadoop.hbase.regionserver.RegionSplitPolicy
402   * @param clazz the class name
403   */
404  public HTableDescriptor setRegionSplitPolicyClassName(String clazz) {
405    getDelegateeForModification().setRegionSplitPolicyClassName(clazz);
406    return this;
407  }
408
409  /**
410   * This gets the class associated with the region split policy which
411   * determines when a region split should occur.  The class used by
412   * default is defined in org.apache.hadoop.hbase.regionserver.RegionSplitPolicy
413   *
414   * @return the class name of the region split policy for this table.
415   * If this returns null, the default split policy is used.
416   */
417  @Override
418   public String getRegionSplitPolicyClassName() {
419    return delegatee.getRegionSplitPolicyClassName();
420  }
421
422  /**
423   * Returns the maximum size upto which a region can grow to after which a region
424   * split is triggered. The region size is represented by the size of the biggest
425   * store file in that region.
426   *
427   * @return max hregion size for table, -1 if not set.
428   *
429   * @see #setMaxFileSize(long)
430   */
431   @Override
432  public long getMaxFileSize() {
433    return delegatee.getMaxFileSize();
434  }
435
436  /**
437   * Sets the maximum size upto which a region can grow to after which a region
438   * split is triggered. The region size is represented by the size of the biggest
439   * store file in that region, i.e. If the biggest store file grows beyond the
440   * maxFileSize, then the region split is triggered. This defaults to a value of
441   * 256 MB.
442   * <p>
443   * This is not an absolute value and might vary. Assume that a single row exceeds
444   * the maxFileSize then the storeFileSize will be greater than maxFileSize since
445   * a single row cannot be split across multiple regions
446   * </p>
447   *
448   * @param maxFileSize The maximum file size that a store file can grow to
449   * before a split is triggered.
450   */
451  public HTableDescriptor setMaxFileSize(long maxFileSize) {
452    getDelegateeForModification().setMaxFileSize(maxFileSize);
453    return this;
454  }
455
456  /**
457   * Returns the size of the memstore after which a flush to filesystem is triggered.
458   *
459   * @return memory cache flush size for each hregion, -1 if not set.
460   *
461   * @see #setMemStoreFlushSize(long)
462   */
463  @Override
464  public long getMemStoreFlushSize() {
465    return delegatee.getMemStoreFlushSize();
466  }
467
468  /**
469   * Represents the maximum size of the memstore after which the contents of the
470   * memstore are flushed to the filesystem. This defaults to a size of 64 MB.
471   *
472   * @param memstoreFlushSize memory cache flush size for each hregion
473   */
474  public HTableDescriptor setMemStoreFlushSize(long memstoreFlushSize) {
475    getDelegateeForModification().setMemStoreFlushSize(memstoreFlushSize);
476    return this;
477  }
478
479  /**
480   * This sets the class associated with the flush policy which determines determines the stores
481   * need to be flushed when flushing a region. The class used by default is defined in
482   * org.apache.hadoop.hbase.regionserver.FlushPolicy.
483   * @param clazz the class name
484   */
485  public HTableDescriptor setFlushPolicyClassName(String clazz) {
486    getDelegateeForModification().setFlushPolicyClassName(clazz);
487    return this;
488  }
489
490  /**
491   * This gets the class associated with the flush policy which determines the stores need to be
492   * flushed when flushing a region. The class used by default is defined in
493   * org.apache.hadoop.hbase.regionserver.FlushPolicy.
494   * @return the class name of the flush policy for this table. If this returns null, the default
495   *         flush policy is used.
496   */
497  @Override
498  public String getFlushPolicyClassName() {
499    return delegatee.getFlushPolicyClassName();
500  }
501
502  /**
503   * Adds a column family.
504   * For the updating purpose please use {@link #modifyFamily(HColumnDescriptor)} instead.
505   * @param family HColumnDescriptor of family to add.
506   */
507  public HTableDescriptor addFamily(final HColumnDescriptor family) {
508    getDelegateeForModification().setColumnFamily(family);
509    return this;
510  }
511
512  /**
513   * Modifies the existing column family.
514   * @param family HColumnDescriptor of family to update
515   * @return this (for chained invocation)
516   */
517  public HTableDescriptor modifyFamily(final HColumnDescriptor family) {
518    getDelegateeForModification().modifyColumnFamily(family);
519    return this;
520  }
521
522  /**
523   * Checks to see if this table contains the given column family
524   * @param familyName Family name or column name.
525   * @return true if the table contains the specified family name
526   */
527  public boolean hasFamily(final byte [] familyName) {
528    return delegatee.hasColumnFamily(familyName);
529  }
530
531  /**
532   * @return Name of this table and then a map of all of the column family
533   * descriptors.
534   * @see #getNameAsString()
535   */
536  @Override
537  public String toString() {
538    return delegatee.toString();
539  }
540
541  /**
542   * @return Name of this table and then a map of all of the column family
543   * descriptors (with only the non-default column family attributes)
544   */
545  @Override
546  public String toStringCustomizedValues() {
547    return delegatee.toStringCustomizedValues();
548  }
549
550  /**
551   * @return map of all table attributes formatted into string.
552   */
553  public String toStringTableAttributes() {
554   return delegatee.toStringTableAttributes();
555  }
556
557  /**
558   * Compare the contents of the descriptor with another one passed as a parameter.
559   * Checks if the obj passed is an instance of HTableDescriptor, if yes then the
560   * contents of the descriptors are compared.
561   *
562   * @return true if the contents of the the two descriptors exactly match
563   *
564   * @see java.lang.Object#equals(java.lang.Object)
565   */
566  @Override
567  public boolean equals(Object obj) {
568    if (this == obj) {
569      return true;
570    }
571    if (obj instanceof HTableDescriptor) {
572      return delegatee.equals(((HTableDescriptor) obj).delegatee);
573    }
574    return false;
575  }
576
577  /**
578   * @see java.lang.Object#hashCode()
579   */
580  @Override
581  public int hashCode() {
582    return delegatee.hashCode();
583  }
584
585  // Comparable
586
587  /**
588   * Compares the descriptor with another descriptor which is passed as a parameter.
589   * This compares the content of the two descriptors and not the reference.
590   *
591   * @return 0 if the contents of the descriptors are exactly matching,
592   *         1 if there is a mismatch in the contents
593   */
594  @Override
595  public int compareTo(final HTableDescriptor other) {
596    return TableDescriptor.COMPARATOR.compare(this, other);
597  }
598
599  /**
600   * Returns an unmodifiable collection of all the {@link HColumnDescriptor}
601   * of all the column families of the table.
602   * @deprecated since 2.0.0 and will be removed in 3.0.0. Use {@link #getColumnFamilies()} instead.
603   * @return Immutable collection of {@link HColumnDescriptor} of all the
604   * column families.
605   * @see #getColumnFamilies()
606   * @see <a href="https://issues.apache.org/jira/browse/HBASE-18008">HBASE-18008</a>
607   */
608  @Deprecated
609  public Collection<HColumnDescriptor> getFamilies() {
610    return Stream.of(delegatee.getColumnFamilies())
611            .map(this::toHColumnDescriptor)
612            .collect(Collectors.toList());
613  }
614
615  /**
616   * Returns the configured replicas per region
617   */
618  @Override
619  public int getRegionReplication() {
620    return delegatee.getRegionReplication();
621  }
622
623  /**
624   * Sets the number of replicas per region.
625   * @param regionReplication the replication factor per region
626   */
627  public HTableDescriptor setRegionReplication(int regionReplication) {
628    getDelegateeForModification().setRegionReplication(regionReplication);
629    return this;
630  }
631
632  /**
633   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0.
634   *             Use {@link #hasRegionMemStoreReplication()} instead
635   */
636  @Deprecated
637  public boolean hasRegionMemstoreReplication() {
638    return hasRegionMemStoreReplication();
639  }
640
641  /**
642   * @return true if the read-replicas memstore replication is enabled.
643   */
644  @Override
645  public boolean hasRegionMemStoreReplication() {
646    return delegatee.hasRegionMemStoreReplication();
647  }
648
649  /**
650   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0.
651   *             Use {@link #setRegionMemStoreReplication(boolean)} instead
652   */
653  @Deprecated
654  public HTableDescriptor setRegionMemstoreReplication(boolean memstoreReplication) {
655    return setRegionMemStoreReplication(memstoreReplication);
656  }
657
658  /**
659   * Enable or Disable the memstore replication from the primary region to the replicas.
660   * The replication will be used only for meta operations (e.g. flush, compaction, ...)
661   *
662   * @param memstoreReplication true if the new data written to the primary region
663   *                                 should be replicated.
664   *                            false if the secondaries can tollerate to have new
665   *                                  data only when the primary flushes the memstore.
666   */
667  public HTableDescriptor setRegionMemStoreReplication(boolean memstoreReplication) {
668    getDelegateeForModification().setRegionMemStoreReplication(memstoreReplication);
669    return this;
670  }
671
672  public HTableDescriptor setPriority(int priority) {
673    getDelegateeForModification().setPriority(priority);
674    return this;
675  }
676
677  @Override
678  public int getPriority() {
679    return delegatee.getPriority();
680  }
681
682  /**
683   * Returns all the column family names of the current table. The map of
684   * HTableDescriptor contains mapping of family name to HColumnDescriptors.
685   * This returns all the keys of the family map which represents the column
686   * family names of the table.
687   *
688   * @return Immutable sorted set of the keys of the families.
689   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
690   *             (<a href="https://issues.apache.org/jira/browse/HBASE-18008">HBASE-18008</a>).
691   *             Use {@link #getColumnFamilyNames()}.
692   */
693  @Deprecated
694  public Set<byte[]> getFamiliesKeys() {
695    return delegatee.getColumnFamilyNames();
696  }
697
698  /**
699   * Returns the count of the column families of the table.
700   *
701   * @return Count of column families of the table
702   */
703  @Override
704  public int getColumnFamilyCount() {
705    return delegatee.getColumnFamilyCount();
706  }
707
708  /**
709   * Returns an array all the {@link HColumnDescriptor} of the column families
710   * of the table.
711   *
712   * @return Array of all the HColumnDescriptors of the current table
713   * @deprecated since 2.0.0 and will be removed in 3.0.0.
714   * @see #getFamilies()
715   * @see <a href="https://issues.apache.org/jira/browse/HBASE-18008">HBASE-18008</a>
716   */
717  @Deprecated
718  @Override
719  public HColumnDescriptor[] getColumnFamilies() {
720    return Stream.of(delegatee.getColumnFamilies())
721            .map(this::toHColumnDescriptor)
722            .toArray(size -> new HColumnDescriptor[size]);
723  }
724
725  /**
726   * Returns the HColumnDescriptor for a specific column family with name as
727   * specified by the parameter column.
728   * @param column Column family name
729   * @return Column descriptor for the passed family name or the family on
730   * passed in column.
731   * @deprecated since 2.0.0 and will be removed in 3.0.0. Use {@link #getColumnFamily(byte[])}
732   *   instead.
733   * @see #getColumnFamily(byte[])
734   * @see <a href="https://issues.apache.org/jira/browse/HBASE-18008">HBASE-18008</a>
735   */
736  @Deprecated
737  public HColumnDescriptor getFamily(final byte[] column) {
738    return toHColumnDescriptor(delegatee.getColumnFamily(column));
739  }
740
741
742  /**
743   * Removes the HColumnDescriptor with name specified by the parameter column
744   * from the table descriptor
745   *
746   * @param column Name of the column family to be removed.
747   * @return Column descriptor for the passed family name or the family on
748   * passed in column.
749   */
750  public HColumnDescriptor removeFamily(final byte [] column) {
751    return toHColumnDescriptor(getDelegateeForModification().removeColumnFamily(column));
752  }
753
754  /**
755   * Return a HColumnDescriptor for user to keep the compatibility as much as possible.
756   * @param desc read-only ColumnFamilyDescriptor
757   * @return The older implementation of ColumnFamilyDescriptor
758   */
759  protected HColumnDescriptor toHColumnDescriptor(ColumnFamilyDescriptor desc) {
760    if (desc == null) {
761      return null;
762    } else if (desc instanceof ModifyableColumnFamilyDescriptor) {
763      return new HColumnDescriptor((ModifyableColumnFamilyDescriptor) desc);
764    } else if (desc instanceof HColumnDescriptor) {
765      return (HColumnDescriptor) desc;
766    } else {
767      return new HColumnDescriptor(new ModifyableColumnFamilyDescriptor(desc));
768    }
769  }
770
771  /**
772   * Add a table coprocessor to this table. The coprocessor
773   * type must be org.apache.hadoop.hbase.coprocessor.RegionCoprocessor.
774   * It won't check if the class can be loaded or not.
775   * Whether a coprocessor is loadable or not will be determined when
776   * a region is opened.
777   * @param className Full class name.
778   * @throws IOException
779   */
780  public HTableDescriptor addCoprocessor(String className) throws IOException {
781    getDelegateeForModification().setCoprocessor(className);
782    return this;
783  }
784
785  /**
786   * Add a table coprocessor to this table. The coprocessor
787   * type must be org.apache.hadoop.hbase.coprocessor.RegionCoprocessor.
788   * It won't check if the class can be loaded or not.
789   * Whether a coprocessor is loadable or not will be determined when
790   * a region is opened.
791   * @param jarFilePath Path of the jar file. If it's null, the class will be
792   * loaded from default classloader.
793   * @param className Full class name.
794   * @param priority Priority
795   * @param kvs Arbitrary key-value parameter pairs passed into the coprocessor.
796   * @throws IOException
797   */
798  public HTableDescriptor addCoprocessor(String className, Path jarFilePath,
799                             int priority, final Map<String, String> kvs)
800  throws IOException {
801    getDelegateeForModification().setCoprocessor(
802      CoprocessorDescriptorBuilder.newBuilder(className)
803        .setJarPath(jarFilePath == null ? null : jarFilePath.toString())
804        .setPriority(priority)
805        .setProperties(kvs == null ? Collections.emptyMap() : kvs)
806        .build());
807    return this;
808  }
809
810  /**
811   * Add a table coprocessor to this table. The coprocessor
812   * type must be org.apache.hadoop.hbase.coprocessor.RegionCoprocessor.
813   * It won't check if the class can be loaded or not.
814   * Whether a coprocessor is loadable or not will be determined when
815   * a region is opened.
816   * @param specStr The Coprocessor specification all in in one String formatted so matches
817   * {@link HConstants#CP_HTD_ATTR_VALUE_PATTERN}
818   * @throws IOException
819   */
820  public HTableDescriptor addCoprocessorWithSpec(final String specStr) throws IOException {
821    getDelegateeForModification().setCoprocessorWithSpec(specStr);
822    return this;
823  }
824
825  /**
826   * Check if the table has an attached co-processor represented by the name className
827   *
828   * @param classNameToMatch - Class name of the co-processor
829   * @return true of the table has a co-processor className
830   */
831  @Override
832  public boolean hasCoprocessor(String classNameToMatch) {
833    return delegatee.hasCoprocessor(classNameToMatch);
834  }
835
836  @Override
837  public Collection<CoprocessorDescriptor> getCoprocessorDescriptors() {
838    return delegatee.getCoprocessorDescriptors();
839  }
840
841  /**
842   * Return the list of attached co-processor represented by their name className
843   *
844   * @return The list of co-processors classNames
845   */
846  public List<String> getCoprocessors() {
847    return getCoprocessorDescriptors().stream().map(CoprocessorDescriptor::getClassName)
848      .collect(Collectors.toList());
849  }
850
851  /**
852   * Remove a coprocessor from those set on the table
853   * @param className Class name of the co-processor
854   */
855  public void removeCoprocessor(String className) {
856    getDelegateeForModification().removeCoprocessor(className);
857  }
858
859  public final static String NAMESPACE_FAMILY_INFO = TableDescriptorBuilder.NAMESPACE_FAMILY_INFO;
860  public final static byte[] NAMESPACE_FAMILY_INFO_BYTES = TableDescriptorBuilder.NAMESPACE_FAMILY_INFO_BYTES;
861  public final static byte[] NAMESPACE_COL_DESC_BYTES = TableDescriptorBuilder.NAMESPACE_COL_DESC_BYTES;
862
863  /** Table descriptor for namespace table */
864  public static final HTableDescriptor NAMESPACE_TABLEDESC
865    = new HTableDescriptor(TableDescriptorBuilder.NAMESPACE_TABLEDESC);
866
867  /**
868   * @deprecated since 0.94.1
869   * @see <a href="https://issues.apache.org/jira/browse/HBASE-6188">HBASE-6188</a>
870   */
871  @Deprecated
872  public HTableDescriptor setOwner(User owner) {
873    getDelegateeForModification().setOwner(owner);
874    return this;
875  }
876
877  /**
878   * @deprecated since 0.94.1
879   * @see <a href="https://issues.apache.org/jira/browse/HBASE-6188">HBASE-6188</a>
880   */
881  // used by admin.rb:alter(table_name,*args) to update owner.
882  @Deprecated
883  public HTableDescriptor setOwnerString(String ownerString) {
884    getDelegateeForModification().setOwnerString(ownerString);
885    return this;
886  }
887
888  /**
889   * @deprecated since 0.94.1
890   * @see <a href="https://issues.apache.org/jira/browse/HBASE-6188">HBASE-6188</a>
891   */
892  @Override
893  @Deprecated
894  public String getOwnerString() {
895    return delegatee.getOwnerString();
896  }
897
898  /**
899   * @return This instance serialized with pb with pb magic prefix
900   * @see #parseFrom(byte[])
901   */
902  public byte[] toByteArray() {
903    return TableDescriptorBuilder.toByteArray(delegatee);
904  }
905
906  /**
907   * @param bytes A pb serialized {@link HTableDescriptor} instance with pb magic prefix
908   * @return An instance of {@link HTableDescriptor} made from <code>bytes</code>
909   * @throws DeserializationException
910   * @throws IOException
911   * @see #toByteArray()
912   */
913  public static HTableDescriptor parseFrom(final byte [] bytes)
914  throws DeserializationException, IOException {
915    TableDescriptor desc = TableDescriptorBuilder.parseFrom(bytes);
916    if (desc instanceof ModifyableTableDescriptor) {
917      return new HTableDescriptor((ModifyableTableDescriptor) desc);
918    } else {
919      return new HTableDescriptor(desc);
920    }
921  }
922
923  /**
924   * Getter for accessing the configuration value by key
925   */
926  public String getConfigurationValue(String key) {
927    return delegatee.getValue(key);
928  }
929
930  /**
931   * Getter for fetching an unmodifiable map.
932   */
933  public Map<String, String> getConfiguration() {
934    return delegatee.getConfiguration();
935  }
936
937  /**
938   * Setter for storing a configuration setting in map.
939   * @param key Config key. Same as XML config key e.g. hbase.something.or.other.
940   * @param value String value. If null, removes the setting.
941   */
942  public HTableDescriptor setConfiguration(String key, String value) {
943    getDelegateeForModification().setValue(key, value);
944    return this;
945  }
946
947  /**
948   * Remove a config setting represented by the key from the map
949   */
950  public void removeConfiguration(final String key) {
951    getDelegateeForModification().removeValue(Bytes.toBytes(key));
952  }
953
954  @Override
955  public Bytes getValue(Bytes key) {
956    return delegatee.getValue(key);
957  }
958
959  @Override
960  public String getValue(String key) {
961    return delegatee.getValue(key);
962  }
963
964  @Override
965  public byte[] getValue(byte[] key) {
966    return delegatee.getValue(key);
967  }
968
969  @Override
970  public Set<byte[]> getColumnFamilyNames() {
971    return delegatee.getColumnFamilyNames();
972  }
973
974  @Override
975  public boolean hasColumnFamily(byte[] name) {
976    return delegatee.hasColumnFamily(name);
977  }
978
979  @Override
980  public ColumnFamilyDescriptor getColumnFamily(byte[] name) {
981    return delegatee.getColumnFamily(name);
982  }
983
984  protected ModifyableTableDescriptor getDelegateeForModification() {
985    return delegatee;
986  }
987
988  @Override
989  public Optional<String> getRegionServerGroup() {
990    return delegatee.getRegionServerGroup();
991  }
992}