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