001/*
002 * Licensed to the Apache Software Foundation (ASF) under one
003 * or more contributor license agreements.  See the NOTICE file
004 * distributed with this work for additional information
005 * regarding copyright ownership.  The ASF licenses this file
006 * to you under the Apache License, Version 2.0 (the
007 * "License"); you may not use this file except in compliance
008 * with the License.  You may obtain a copy of the License at
009 *
010 *     http://www.apache.org/licenses/LICENSE-2.0
011 *
012 * Unless required by applicable law or agreed to in writing, software
013 * distributed under the License is distributed on an "AS IS" BASIS,
014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
015 * See the License for the specific language governing permissions and
016 * limitations under the License.
017 */
018package org.apache.hadoop.hbase;
019
020import java.nio.ByteBuffer;
021import java.nio.charset.StandardCharsets;
022import java.util.Arrays;
023import java.util.Set;
024import java.util.concurrent.CopyOnWriteArraySet;
025import org.apache.commons.lang3.ArrayUtils;
026import org.apache.hadoop.hbase.util.Bytes;
027import org.apache.yetus.audience.InterfaceAudience;
028
029import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
030
031/**
032 * Immutable POJO class for representing a table name. Which is of the form: <table
033 * namespace>:<table qualifier> Two special namespaces: 1. hbase - system namespace, used
034 * to contain hbase internal tables 2. default - tables with no explicit specified namespace will
035 * automatically fall into this namespace. ie a) foo:bar, means namespace=foo and qualifier=bar b)
036 * bar, means namespace=default and qualifier=bar c) default:bar, means namespace=default and
037 * qualifier=bar
038 * <p>
039 * Internally, in this class, we cache the instances to limit the number of objects and make the
040 * "equals" faster. We try to minimize the number of objects created of the number of array copy to
041 * check if we already have an instance of this TableName. The code is not optimize for a new
042 * instance creation but is optimized to check for existence.
043 * </p>
044 */
045@InterfaceAudience.Public
046public final class TableName implements Comparable<TableName> {
047
048  /** See {@link #createTableNameIfNecessary(ByteBuffer, ByteBuffer)} */
049  private static final Set<TableName> tableCache = new CopyOnWriteArraySet<>();
050
051  /** Namespace delimiter */
052  // this should always be only 1 byte long
053  public final static char NAMESPACE_DELIM = ':';
054
055  // A non-capture group so that this can be embedded.
056  // regex is a bit more complicated to support nuance of tables
057  // in default namespace
058  // Allows only letters, digits and '_'
059  public static final String VALID_NAMESPACE_REGEX = "(?:[_\\p{Digit}\\p{IsAlphabetic}]+)";
060  // Allows only letters, digits, '_', '-' and '.'
061  public static final String VALID_TABLE_QUALIFIER_REGEX =
062    "(?:[_\\p{Digit}\\p{IsAlphabetic}][-_.\\p{Digit}\\p{IsAlphabetic}]*)";
063  // Concatenation of NAMESPACE_REGEX and TABLE_QUALIFIER_REGEX,
064  // with NAMESPACE_DELIM as delimiter
065  public static final String VALID_USER_TABLE_REGEX = "(?:(?:(?:" + VALID_NAMESPACE_REGEX + "\\"
066    + NAMESPACE_DELIM + ")?)" + "(?:" + VALID_TABLE_QUALIFIER_REGEX + "))";
067
068  /** The hbase:meta table's name. */
069  public static final TableName META_TABLE_NAME =
070    valueOf(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR, "meta");
071
072  /**
073   * The Namespace table's name.
074   * @deprecated since 3.0.0 and will be removed in 4.0.0. We have folded the data in namespace
075   *             table into meta table, so do not use it any more.
076   * @see <a href="https://issues.apache.org/jira/browse/HBASE-21154">HBASE-21154</a>
077   */
078  @Deprecated
079  public static final TableName NAMESPACE_TABLE_NAME =
080    valueOf(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR, "namespace");
081
082  public static final String OLD_META_STR = ".META.";
083  public static final String OLD_ROOT_STR = "-ROOT-";
084
085  /** One globally disallowed name */
086  public static final String DISALLOWED_TABLE_NAME = "zookeeper";
087
088  /** Returns True if <code>tn</code> is the hbase:meta table name. */
089  public static boolean isMetaTableName(final TableName tn) {
090    return tn.equals(TableName.META_TABLE_NAME);
091  }
092
093  /**
094   * TableName for old -ROOT- table. It is used to read/process old WALs which have ROOT edits.
095   */
096  public static final TableName OLD_ROOT_TABLE_NAME = getADummyTableName(OLD_ROOT_STR);
097  /**
098   * TableName for old .META. table. Used in testing.
099   */
100  public static final TableName OLD_META_TABLE_NAME = getADummyTableName(OLD_META_STR);
101
102  private final byte[] name;
103  private final String nameAsString;
104  private final byte[] namespace;
105  private final String namespaceAsString;
106  private final byte[] qualifier;
107  private final String qualifierAsString;
108  private final boolean systemTable;
109  private final int hashCode;
110
111  /**
112   * Check passed byte array, "tableName", is legal user-space table name.
113   * @return Returns passed <code>tableName</code> param
114   * @throws IllegalArgumentException if passed a tableName is null or is made of other than 'word'
115   *                                  characters or underscores: i.e.
116   *                                  <code>[\p{IsAlphabetic}\p{Digit}.-:]</code>. The ':' is used
117   *                                  to delimit the namespace from the table name and can be used
118   *                                  for nothing else. Namespace names can only contain 'word'
119   *                                  characters <code>[\p{IsAlphabetic}\p{Digit}]</code> or '_'
120   *                                  Qualifier names can only contain 'word' characters
121   *                                  <code>[\p{IsAlphabetic}\p{Digit}]</code> or '_', '.' or '-'.
122   *                                  The name may not start with '.' or '-'. Valid fully qualified
123   *                                  table names: foo:bar, namespace=&gt;foo, table=&gt;bar
124   *                                  org:foo.bar, namespace=org, table=&gt;foo.bar
125   */
126  public static byte[] isLegalFullyQualifiedTableName(final byte[] tableName) {
127    if (tableName == null || tableName.length <= 0) {
128      throw new IllegalArgumentException("Name is null or empty");
129    }
130
131    int namespaceDelimIndex = ArrayUtils.lastIndexOf(tableName, (byte) NAMESPACE_DELIM);
132    if (namespaceDelimIndex < 0) {
133      isLegalTableQualifierName(tableName);
134    } else {
135      isLegalNamespaceName(tableName, 0, namespaceDelimIndex);
136      isLegalTableQualifierName(tableName, namespaceDelimIndex + 1, tableName.length);
137    }
138    return tableName;
139  }
140
141  public static byte[] isLegalTableQualifierName(final byte[] qualifierName) {
142    isLegalTableQualifierName(qualifierName, 0, qualifierName.length, false);
143    return qualifierName;
144  }
145
146  public static byte[] isLegalTableQualifierName(final byte[] qualifierName, boolean isSnapshot) {
147    isLegalTableQualifierName(qualifierName, 0, qualifierName.length, isSnapshot);
148    return qualifierName;
149  }
150
151  /**
152   * Qualifier names can only contain 'word' characters <code>[\p{IsAlphabetic}\p{Digit}]</code> or
153   * '_', '.' or '-'. The name may not start with '.' or '-'.
154   * @param qualifierName byte array containing the qualifier name
155   * @param start         start index
156   * @param end           end index (exclusive)
157   */
158  public static void isLegalTableQualifierName(final byte[] qualifierName, int start, int end) {
159    isLegalTableQualifierName(qualifierName, start, end, false);
160  }
161
162  public static void isLegalTableQualifierName(final byte[] qualifierName, int start, int end,
163    boolean isSnapshot) {
164    if (end - start < 1) {
165      throw new IllegalArgumentException(
166        isSnapshot ? "Snapshot" : "Table" + " qualifier must not be empty");
167    }
168    String qualifierString = Bytes.toString(qualifierName, start, end - start);
169    if (qualifierName[start] == '.' || qualifierName[start] == '-') {
170      throw new IllegalArgumentException("Illegal first character <" + qualifierName[start]
171        + "> at 0. " + (isSnapshot ? "Snapshot" : "User-space table")
172        + " qualifiers can only start with 'alphanumeric " + "characters' from any language: "
173        + qualifierString);
174    }
175    if (qualifierString.equals(DISALLOWED_TABLE_NAME)) {
176      // Per https://zookeeper.apache.org/doc/r3.4.10/zookeeperProgrammers.html#ch_zkDataModel
177      // A znode named "zookeeper" is disallowed by zookeeper.
178      throw new IllegalArgumentException("Tables may not be named '" + DISALLOWED_TABLE_NAME + "'");
179    }
180    for (int i = 0; i < qualifierString.length(); i++) {
181      // Treat the string as a char-array as some characters may be multi-byte
182      char c = qualifierString.charAt(i);
183      // Check for letter, digit, underscore, hyphen, or period, and allowed by ZK.
184      // ZooKeeper also has limitations, but Character.isAlphabetic omits those all
185      // See https://zookeeper.apache.org/doc/r3.4.10/zookeeperProgrammers.html#ch_zkDataModel
186      if (Character.isAlphabetic(c) || Character.isDigit(c) || c == '_' || c == '-' || c == '.') {
187        continue;
188      }
189      throw new IllegalArgumentException("Illegal character code:" + (int) c + ", <" + c + "> at "
190        + i + ". " + (isSnapshot ? "Snapshot" : "User-space table")
191        + " qualifiers may only contain 'alphanumeric characters' and digits: " + qualifierString);
192    }
193  }
194
195  public static void isLegalNamespaceName(byte[] namespaceName) {
196    isLegalNamespaceName(namespaceName, 0, namespaceName.length);
197  }
198
199  /**
200   * Valid namespace characters are alphabetic characters, numbers, and underscores.
201   */
202  public static void isLegalNamespaceName(final byte[] namespaceName, final int start,
203    final int end) {
204    if (end - start < 1) {
205      throw new IllegalArgumentException("Namespace name must not be empty");
206    }
207    String nsString = new String(namespaceName, start, (end - start), StandardCharsets.UTF_8);
208    if (nsString.equals(DISALLOWED_TABLE_NAME)) {
209      // Per https://zookeeper.apache.org/doc/r3.4.10/zookeeperProgrammers.html#ch_zkDataModel
210      // A znode named "zookeeper" is disallowed by zookeeper.
211      throw new IllegalArgumentException("Tables may not be named '" + DISALLOWED_TABLE_NAME + "'");
212    }
213    for (int i = 0; i < nsString.length(); i++) {
214      // Treat the string as a char-array as some characters may be multi-byte
215      char c = nsString.charAt(i);
216      // ZooKeeper also has limitations, but Character.isAlphabetic omits those all
217      // See https://zookeeper.apache.org/doc/r3.4.10/zookeeperProgrammers.html#ch_zkDataModel
218      if (Character.isAlphabetic(c) || Character.isDigit(c) || c == '_') {
219        continue;
220      }
221      throw new IllegalArgumentException(
222        "Illegal character <" + c + "> at " + i + ". Namespaces may only contain "
223          + "'alphanumeric characters' from any language and digits: " + nsString);
224    }
225  }
226
227  public byte[] getName() {
228    return name;
229  }
230
231  public String getNameAsString() {
232    return nameAsString;
233  }
234
235  public byte[] getNamespace() {
236    return namespace;
237  }
238
239  public String getNamespaceAsString() {
240    return namespaceAsString;
241  }
242
243  /**
244   * Ideally, getNameAsString should contain namespace within it, but if the namespace is default,
245   * it just returns the name. This method takes care of this corner case.
246   */
247  public String getNameWithNamespaceInclAsString() {
248    if (getNamespaceAsString().equals(NamespaceDescriptor.DEFAULT_NAMESPACE_NAME_STR)) {
249      return NamespaceDescriptor.DEFAULT_NAMESPACE_NAME_STR + TableName.NAMESPACE_DELIM
250        + getNameAsString();
251    }
252    return getNameAsString();
253  }
254
255  public byte[] getQualifier() {
256    return qualifier;
257  }
258
259  public String getQualifierAsString() {
260    return qualifierAsString;
261  }
262
263  /** Returns A pointer to TableName as String bytes. */
264  public byte[] toBytes() {
265    return name;
266  }
267
268  public boolean isSystemTable() {
269    return systemTable;
270  }
271
272  @Override
273  public String toString() {
274    return nameAsString;
275  }
276
277  private TableName(ByteBuffer namespace, ByteBuffer qualifier) throws IllegalArgumentException {
278    this.qualifier = new byte[qualifier.remaining()];
279    qualifier.duplicate().get(this.qualifier);
280    this.qualifierAsString = Bytes.toString(this.qualifier);
281
282    if (qualifierAsString.equals(OLD_ROOT_STR)) {
283      throw new IllegalArgumentException(OLD_ROOT_STR + " has been deprecated.");
284    }
285    if (qualifierAsString.equals(OLD_META_STR)) {
286      throw new IllegalArgumentException(
287        OLD_META_STR + " no longer exists. The table has been " + "renamed to " + META_TABLE_NAME);
288    }
289
290    if (Bytes.equals(NamespaceDescriptor.DEFAULT_NAMESPACE_NAME, namespace)) {
291      // Using the same objects: this will make the comparison faster later
292      this.namespace = NamespaceDescriptor.DEFAULT_NAMESPACE_NAME;
293      this.namespaceAsString = NamespaceDescriptor.DEFAULT_NAMESPACE_NAME_STR;
294      this.systemTable = false;
295
296      // The name does not include the namespace when it's the default one.
297      this.nameAsString = qualifierAsString;
298      this.name = this.qualifier;
299    } else {
300      if (Bytes.equals(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME, namespace)) {
301        this.namespace = NamespaceDescriptor.SYSTEM_NAMESPACE_NAME;
302        this.namespaceAsString = NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR;
303        this.systemTable = true;
304      } else {
305        this.namespace = new byte[namespace.remaining()];
306        namespace.duplicate().get(this.namespace);
307        this.namespaceAsString = Bytes.toString(this.namespace);
308        this.systemTable = false;
309      }
310      this.nameAsString = namespaceAsString + NAMESPACE_DELIM + qualifierAsString;
311      this.name = Bytes.toBytes(nameAsString);
312    }
313
314    this.hashCode = nameAsString.hashCode();
315
316    isLegalNamespaceName(this.namespace);
317    isLegalTableQualifierName(this.qualifier);
318  }
319
320  /** This is only for the old and meta tables. */
321  private TableName(String qualifier) {
322    this.qualifier = Bytes.toBytes(qualifier);
323    this.qualifierAsString = qualifier;
324
325    this.namespace = NamespaceDescriptor.SYSTEM_NAMESPACE_NAME;
326    this.namespaceAsString = NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR;
327    this.systemTable = true;
328
329    // WARNING: nameAsString is different than name for old meta & root!
330    // This is by design.
331    this.nameAsString = namespaceAsString + NAMESPACE_DELIM + qualifierAsString;
332    this.name = this.qualifier;
333
334    this.hashCode = nameAsString.hashCode();
335  }
336
337  /**
338   * Check that the object does not exist already. There are two reasons for creating the objects
339   * only once: 1) With 100K regions, the table names take ~20MB. 2) Equals becomes much faster as
340   * it's resolved with a reference and an int comparison.
341   */
342  private static TableName createTableNameIfNecessary(ByteBuffer bns, ByteBuffer qns) {
343    for (TableName tn : tableCache) {
344      if (Bytes.equals(tn.getQualifier(), qns) && Bytes.equals(tn.getNamespace(), bns)) {
345        return tn;
346      }
347    }
348
349    TableName newTable = new TableName(bns, qns);
350    if (tableCache.add(newTable)) { // Adds the specified element if it is not already present
351      return newTable;
352    }
353
354    // Someone else added it. Let's find it.
355    for (TableName tn : tableCache) {
356      if (Bytes.equals(tn.getQualifier(), qns) && Bytes.equals(tn.getNamespace(), bns)) {
357        return tn;
358      }
359    }
360    // this should never happen.
361    throw new IllegalStateException(newTable + " was supposed to be in the cache");
362  }
363
364  /**
365   * It is used to create table names for old META, and ROOT table. These tables are not really
366   * legal tables. They are not added into the cache.
367   * @return a dummy TableName instance (with no validation) for the passed qualifier
368   */
369  private static TableName getADummyTableName(String qualifier) {
370    return new TableName(qualifier);
371  }
372
373  public static TableName valueOf(String namespaceAsString, String qualifierAsString) {
374    if (namespaceAsString == null || namespaceAsString.length() < 1) {
375      namespaceAsString = NamespaceDescriptor.DEFAULT_NAMESPACE_NAME_STR;
376    }
377
378    for (TableName tn : tableCache) {
379      if (
380        qualifierAsString.equals(tn.getQualifierAsString())
381          && namespaceAsString.equals(tn.getNamespaceAsString())
382      ) {
383        return tn;
384      }
385    }
386
387    return createTableNameIfNecessary(ByteBuffer.wrap(Bytes.toBytes(namespaceAsString)),
388      ByteBuffer.wrap(Bytes.toBytes(qualifierAsString)));
389  }
390
391  /**
392   * Construct a TableName
393   * @param fullName will use the entire byte array
394   * @throws IllegalArgumentException if fullName equals old root or old meta. Some code depends on
395   *                                  this. The test is buried in the table creation to save on
396   *                                  array comparison when we're creating a standard table object
397   *                                  that will be in the cache.
398   */
399  public static TableName valueOf(byte[] fullName) throws IllegalArgumentException {
400    return valueOf(fullName, 0, fullName.length);
401  }
402
403  /**
404   * Construct a TableName
405   * @param fullName byte array to look into
406   * @param offset   within said array
407   * @param length   within said array
408   * @throws IllegalArgumentException if fullName equals old root or old meta.
409   */
410  public static TableName valueOf(byte[] fullName, int offset, int length)
411    throws IllegalArgumentException {
412    Preconditions.checkArgument(offset >= 0, "offset must be non-negative but was %s", offset);
413    Preconditions.checkArgument(offset < fullName.length, "offset (%s) must be < array length (%s)",
414      offset, fullName.length);
415    Preconditions.checkArgument(length <= fullName.length,
416      "length (%s) must be <= array length (%s)", length, fullName.length);
417    for (TableName tn : tableCache) {
418      final byte[] tnName = tn.getName();
419      if (Bytes.equals(tnName, 0, tnName.length, fullName, offset, length)) {
420        return tn;
421      }
422    }
423
424    int namespaceDelimIndex =
425      ArrayUtils.lastIndexOf(fullName, (byte) NAMESPACE_DELIM, offset + length - 1);
426
427    if (namespaceDelimIndex < offset) {
428      return createTableNameIfNecessary(ByteBuffer.wrap(NamespaceDescriptor.DEFAULT_NAMESPACE_NAME),
429        ByteBuffer.wrap(fullName, offset, length));
430    } else {
431      return createTableNameIfNecessary(ByteBuffer.wrap(fullName, offset, namespaceDelimIndex),
432        ByteBuffer.wrap(fullName, namespaceDelimIndex + 1, length - (namespaceDelimIndex + 1)));
433    }
434  }
435
436  /**
437   * Construct a TableName
438   * @param fullname of a table, possibly with a leading namespace and ':' as delimiter.
439   * @throws IllegalArgumentException if fullName equals old root or old meta.
440   */
441  public static TableName valueOf(ByteBuffer fullname) {
442    fullname = fullname.duplicate();
443    fullname.mark();
444    boolean miss = true;
445    while (fullname.hasRemaining() && miss) {
446      miss = ((byte) NAMESPACE_DELIM) != fullname.get();
447    }
448    if (miss) {
449      fullname.reset();
450      return valueOf(null, fullname);
451    } else {
452      ByteBuffer qualifier = fullname.slice();
453      int delimiterIndex = fullname.position() - 1;
454      fullname.reset();
455      // changing variable name for clarity
456      ByteBuffer namespace = fullname.duplicate();
457      namespace.limit(delimiterIndex);
458      return valueOf(namespace, qualifier);
459    }
460  }
461
462  /**
463   * Construct a TableName
464   * @throws IllegalArgumentException if fullName equals old root or old meta. Some code depends on
465   *                                  this.
466   */
467  public static TableName valueOf(String name) {
468    for (TableName tn : tableCache) {
469      if (name.equals(tn.getNameAsString())) {
470        return tn;
471      }
472    }
473
474    final int namespaceDelimIndex = name.indexOf(NAMESPACE_DELIM);
475
476    if (namespaceDelimIndex < 0) {
477      return createTableNameIfNecessary(ByteBuffer.wrap(NamespaceDescriptor.DEFAULT_NAMESPACE_NAME),
478        ByteBuffer.wrap(Bytes.toBytes(name)));
479    } else {
480      // indexOf is by character, not byte (consider multi-byte characters)
481      String ns = name.substring(0, namespaceDelimIndex);
482      String qualifier = name.substring(namespaceDelimIndex + 1);
483      return createTableNameIfNecessary(ByteBuffer.wrap(Bytes.toBytes(ns)),
484        ByteBuffer.wrap(Bytes.toBytes(qualifier)));
485    }
486  }
487
488  public static TableName valueOf(byte[] namespace, byte[] qualifier) {
489    if (namespace == null || namespace.length < 1) {
490      namespace = NamespaceDescriptor.DEFAULT_NAMESPACE_NAME;
491    }
492
493    for (TableName tn : tableCache) {
494      if (
495        Arrays.equals(tn.getQualifier(), qualifier) && Arrays.equals(tn.getNamespace(), namespace)
496      ) {
497        return tn;
498      }
499    }
500
501    return createTableNameIfNecessary(ByteBuffer.wrap(namespace), ByteBuffer.wrap(qualifier));
502  }
503
504  public static TableName valueOf(ByteBuffer namespace, ByteBuffer qualifier) {
505    if (namespace == null || namespace.remaining() < 1) {
506      return createTableNameIfNecessary(ByteBuffer.wrap(NamespaceDescriptor.DEFAULT_NAMESPACE_NAME),
507        qualifier);
508    }
509
510    return createTableNameIfNecessary(namespace, qualifier);
511  }
512
513  @Override
514  public boolean equals(Object o) {
515    if (this == o) return true;
516    if (o == null || getClass() != o.getClass()) return false;
517
518    TableName tableName = (TableName) o;
519
520    return o.hashCode() == hashCode && nameAsString.equals(tableName.nameAsString);
521  }
522
523  @Override
524  public int hashCode() {
525    return hashCode;
526  }
527
528  @Override
529  public int compareTo(TableName tableName) {
530    // For performance reasons, the ordering is not lexicographic.
531    if (this == tableName) {
532      return 0;
533    }
534    if (this.hashCode < tableName.hashCode()) {
535      return -1;
536    }
537    if (this.hashCode > tableName.hashCode()) {
538      return 1;
539    }
540    return this.nameAsString.compareTo(tableName.getNameAsString());
541  }
542
543}