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