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  /** The Namespace table's name. */
073  public static final TableName NAMESPACE_TABLE_NAME =
074    valueOf(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR, "namespace");
075
076  public static final String OLD_META_STR = ".META.";
077  public static final String OLD_ROOT_STR = "-ROOT-";
078
079  /** One globally disallowed name */
080  public static final String DISALLOWED_TABLE_NAME = "zookeeper";
081
082  /** Returns True if <code>tn</code> is the hbase:meta table name. */
083  public static boolean isMetaTableName(final TableName tn) {
084    return tn.equals(TableName.META_TABLE_NAME);
085  }
086
087  /**
088   * TableName for old -ROOT- table. It is used to read/process old WALs which have ROOT edits.
089   */
090  public static final TableName OLD_ROOT_TABLE_NAME = getADummyTableName(OLD_ROOT_STR);
091  /**
092   * TableName for old .META. table. Used in testing.
093   */
094  public static final TableName OLD_META_TABLE_NAME = getADummyTableName(OLD_META_STR);
095
096  private final byte[] name;
097  private final String nameAsString;
098  private final byte[] namespace;
099  private final String namespaceAsString;
100  private final byte[] qualifier;
101  private final String qualifierAsString;
102  private final boolean systemTable;
103  private final int hashCode;
104
105  /**
106   * Check passed byte array, "tableName", is legal user-space table name.
107   * @return Returns passed <code>tableName</code> param
108   * @throws IllegalArgumentException if passed a tableName is null or is made of other than 'word'
109   *                                  characters or underscores: i.e.
110   *                                  <code>[\p{IsAlphabetic}\p{Digit}.-:]</code>. The ':' is used
111   *                                  to delimit the namespace from the table name and can be used
112   *                                  for nothing else. Namespace names can only contain 'word'
113   *                                  characters <code>[\p{IsAlphabetic}\p{Digit}]</code> or '_'
114   *                                  Qualifier names can only contain 'word' characters
115   *                                  <code>[\p{IsAlphabetic}\p{Digit}]</code> or '_', '.' or '-'.
116   *                                  The name may not start with '.' or '-'. Valid fully qualified
117   *                                  table names: foo:bar, namespace=&gt;foo, table=&gt;bar
118   *                                  org:foo.bar, namespace=org, table=&gt;foo.bar
119   */
120  public static byte[] isLegalFullyQualifiedTableName(final byte[] tableName) {
121    if (tableName == null || tableName.length <= 0) {
122      throw new IllegalArgumentException("Name is null or empty");
123    }
124
125    int namespaceDelimIndex = org.apache.hbase.thirdparty.com.google.common.primitives.Bytes
126      .lastIndexOf(tableName, (byte) NAMESPACE_DELIM);
127    if (namespaceDelimIndex < 0) {
128      isLegalTableQualifierName(tableName);
129    } else {
130      isLegalNamespaceName(tableName, 0, namespaceDelimIndex);
131      isLegalTableQualifierName(tableName, namespaceDelimIndex + 1, tableName.length);
132    }
133    return tableName;
134  }
135
136  public static byte[] isLegalTableQualifierName(final byte[] qualifierName) {
137    isLegalTableQualifierName(qualifierName, 0, qualifierName.length, false);
138    return qualifierName;
139  }
140
141  public static byte[] isLegalTableQualifierName(final byte[] qualifierName, boolean isSnapshot) {
142    isLegalTableQualifierName(qualifierName, 0, qualifierName.length, isSnapshot);
143    return qualifierName;
144  }
145
146  /**
147   * Qualifier names can only contain 'word' characters <code>[\p{IsAlphabetic}\p{Digit}]</code> or
148   * '_', '.' or '-'. The name may not start with '.' or '-'.
149   * @param qualifierName byte array containing the qualifier name
150   * @param start         start index
151   * @param end           end index (exclusive)
152   */
153  public static void isLegalTableQualifierName(final byte[] qualifierName, int start, int end) {
154    isLegalTableQualifierName(qualifierName, start, end, false);
155  }
156
157  public static void isLegalTableQualifierName(final byte[] qualifierName, int start, int end,
158    boolean isSnapshot) {
159    if (end - start < 1) {
160      throw new IllegalArgumentException(
161        isSnapshot ? "Snapshot" : "Table" + " qualifier must not be empty");
162    }
163    String qualifierString = Bytes.toString(qualifierName, start, end - start);
164    if (qualifierName[start] == '.' || qualifierName[start] == '-') {
165      throw new IllegalArgumentException("Illegal first character <" + qualifierName[start]
166        + "> at 0. " + (isSnapshot ? "Snapshot" : "User-space table")
167        + " qualifiers can only start with 'alphanumeric " + "characters' from any language: "
168        + qualifierString);
169    }
170    if (qualifierString.equals(DISALLOWED_TABLE_NAME)) {
171      // Per https://zookeeper.apache.org/doc/r3.4.10/zookeeperProgrammers.html#ch_zkDataModel
172      // A znode named "zookeeper" is disallowed by zookeeper.
173      throw new IllegalArgumentException("Tables may not be named '" + DISALLOWED_TABLE_NAME + "'");
174    }
175    for (int i = 0; i < qualifierString.length(); i++) {
176      // Treat the string as a char-array as some characters may be multi-byte
177      char c = qualifierString.charAt(i);
178      // Check for letter, digit, underscore, hyphen, or period, and allowed by ZK.
179      // ZooKeeper also has limitations, but Character.isAlphabetic omits those all
180      // See https://zookeeper.apache.org/doc/r3.4.10/zookeeperProgrammers.html#ch_zkDataModel
181      if (Character.isAlphabetic(c) || Character.isDigit(c) || c == '_' || c == '-' || c == '.') {
182        continue;
183      }
184      throw new IllegalArgumentException("Illegal character code:" + (int) c + ", <" + c + "> at "
185        + i + ". " + (isSnapshot ? "Snapshot" : "User-space table")
186        + " qualifiers may only contain 'alphanumeric characters' and digits: " + qualifierString);
187    }
188  }
189
190  public static void isLegalNamespaceName(byte[] namespaceName) {
191    isLegalNamespaceName(namespaceName, 0, namespaceName.length);
192  }
193
194  /**
195   * Valid namespace characters are alphabetic characters, numbers, and underscores.
196   */
197  public static void isLegalNamespaceName(final byte[] namespaceName, final int start,
198    final int end) {
199    if (end - start < 1) {
200      throw new IllegalArgumentException("Namespace name must not be empty");
201    }
202    String nsString = new String(namespaceName, start, (end - start), StandardCharsets.UTF_8);
203    if (nsString.equals(DISALLOWED_TABLE_NAME)) {
204      // Per https://zookeeper.apache.org/doc/r3.4.10/zookeeperProgrammers.html#ch_zkDataModel
205      // A znode named "zookeeper" is disallowed by zookeeper.
206      throw new IllegalArgumentException("Tables may not be named '" + DISALLOWED_TABLE_NAME + "'");
207    }
208    for (int i = 0; i < nsString.length(); i++) {
209      // Treat the string as a char-array as some characters may be multi-byte
210      char c = nsString.charAt(i);
211      // ZooKeeper also has limitations, but Character.isAlphabetic omits those all
212      // See https://zookeeper.apache.org/doc/r3.4.10/zookeeperProgrammers.html#ch_zkDataModel
213      if (Character.isAlphabetic(c) || Character.isDigit(c) || c == '_') {
214        continue;
215      }
216      throw new IllegalArgumentException(
217        "Illegal character <" + c + "> at " + i + ". Namespaces may only contain "
218          + "'alphanumeric characters' from any language and digits: " + nsString);
219    }
220  }
221
222  public byte[] getName() {
223    return name;
224  }
225
226  public String getNameAsString() {
227    return nameAsString;
228  }
229
230  public byte[] getNamespace() {
231    return namespace;
232  }
233
234  public String getNamespaceAsString() {
235    return namespaceAsString;
236  }
237
238  /**
239   * Ideally, getNameAsString should contain namespace within it, but if the namespace is default,
240   * it just returns the name. This method takes care of this corner case.
241   */
242  public String getNameWithNamespaceInclAsString() {
243    if (getNamespaceAsString().equals(NamespaceDescriptor.DEFAULT_NAMESPACE_NAME_STR)) {
244      return NamespaceDescriptor.DEFAULT_NAMESPACE_NAME_STR + TableName.NAMESPACE_DELIM
245        + getNameAsString();
246    }
247    return getNameAsString();
248  }
249
250  public byte[] getQualifier() {
251    return qualifier;
252  }
253
254  public String getQualifierAsString() {
255    return qualifierAsString;
256  }
257
258  /** Returns A pointer to TableName as String bytes. */
259  public byte[] toBytes() {
260    return name;
261  }
262
263  public boolean isSystemTable() {
264    return systemTable;
265  }
266
267  @Override
268  public String toString() {
269    return nameAsString;
270  }
271
272  private TableName(ByteBuffer namespace, ByteBuffer qualifier) throws IllegalArgumentException {
273    this.qualifier = new byte[qualifier.remaining()];
274    qualifier.duplicate().get(this.qualifier);
275    this.qualifierAsString = Bytes.toString(this.qualifier);
276
277    if (qualifierAsString.equals(OLD_ROOT_STR)) {
278      throw new IllegalArgumentException(OLD_ROOT_STR + " has been deprecated.");
279    }
280    if (qualifierAsString.equals(OLD_META_STR)) {
281      throw new IllegalArgumentException(
282        OLD_META_STR + " no longer exists. The table has been " + "renamed to " + META_TABLE_NAME);
283    }
284
285    if (Bytes.equals(NamespaceDescriptor.DEFAULT_NAMESPACE_NAME, namespace)) {
286      // Using the same objects: this will make the comparison faster later
287      this.namespace = NamespaceDescriptor.DEFAULT_NAMESPACE_NAME;
288      this.namespaceAsString = NamespaceDescriptor.DEFAULT_NAMESPACE_NAME_STR;
289      this.systemTable = false;
290
291      // The name does not include the namespace when it's the default one.
292      this.nameAsString = qualifierAsString;
293      this.name = this.qualifier;
294    } else {
295      if (Bytes.equals(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME, namespace)) {
296        this.namespace = NamespaceDescriptor.SYSTEM_NAMESPACE_NAME;
297        this.namespaceAsString = NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR;
298        this.systemTable = true;
299      } else {
300        this.namespace = new byte[namespace.remaining()];
301        namespace.duplicate().get(this.namespace);
302        this.namespaceAsString = Bytes.toString(this.namespace);
303        this.systemTable = false;
304      }
305      this.nameAsString = namespaceAsString + NAMESPACE_DELIM + qualifierAsString;
306      this.name = Bytes.toBytes(nameAsString);
307    }
308
309    this.hashCode = nameAsString.hashCode();
310
311    isLegalNamespaceName(this.namespace);
312    isLegalTableQualifierName(this.qualifier);
313  }
314
315  /** This is only for the old and meta tables. */
316  private TableName(String qualifier) {
317    this.qualifier = Bytes.toBytes(qualifier);
318    this.qualifierAsString = qualifier;
319
320    this.namespace = NamespaceDescriptor.SYSTEM_NAMESPACE_NAME;
321    this.namespaceAsString = NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR;
322    this.systemTable = true;
323
324    // WARNING: nameAsString is different than name for old meta & root!
325    // This is by design.
326    this.nameAsString = namespaceAsString + NAMESPACE_DELIM + qualifierAsString;
327    this.name = this.qualifier;
328
329    this.hashCode = nameAsString.hashCode();
330  }
331
332  /**
333   * Check that the object does not exist already. There are two reasons for creating the objects
334   * only once: 1) With 100K regions, the table names take ~20MB. 2) Equals becomes much faster as
335   * it's resolved with a reference and an int comparison.
336   */
337  private static TableName createTableNameIfNecessary(ByteBuffer bns, ByteBuffer qns) {
338    for (TableName tn : tableCache) {
339      if (Bytes.equals(tn.getQualifier(), qns) && Bytes.equals(tn.getNamespace(), bns)) {
340        return tn;
341      }
342    }
343
344    TableName newTable = new TableName(bns, qns);
345    if (tableCache.add(newTable)) { // Adds the specified element if it is not already present
346      return newTable;
347    }
348
349    // Someone else added it. Let's find it.
350    for (TableName tn : tableCache) {
351      if (Bytes.equals(tn.getQualifier(), qns) && Bytes.equals(tn.getNamespace(), bns)) {
352        return tn;
353      }
354    }
355    // this should never happen.
356    throw new IllegalStateException(newTable + " was supposed to be in the cache");
357  }
358
359  /**
360   * It is used to create table names for old META, and ROOT table. These tables are not really
361   * legal tables. They are not added into the cache.
362   * @return a dummy TableName instance (with no validation) for the passed qualifier
363   */
364  private static TableName getADummyTableName(String qualifier) {
365    return new TableName(qualifier);
366  }
367
368  public static TableName valueOf(String namespaceAsString, String qualifierAsString) {
369    if (namespaceAsString == null || namespaceAsString.length() < 1) {
370      namespaceAsString = NamespaceDescriptor.DEFAULT_NAMESPACE_NAME_STR;
371    }
372
373    for (TableName tn : tableCache) {
374      if (
375        qualifierAsString.equals(tn.getQualifierAsString())
376          && namespaceAsString.equals(tn.getNamespaceAsString())
377      ) {
378        return tn;
379      }
380    }
381
382    return createTableNameIfNecessary(ByteBuffer.wrap(Bytes.toBytes(namespaceAsString)),
383      ByteBuffer.wrap(Bytes.toBytes(qualifierAsString)));
384  }
385
386  /**
387   * Construct a TableName
388   * @param fullName will use the entire byte array
389   * @throws IllegalArgumentException if fullName equals old root or old meta. Some code depends on
390   *                                  this. The test is buried in the table creation to save on
391   *                                  array comparison when we're creating a standard table object
392   *                                  that will be in the cache.
393   */
394  public static TableName valueOf(byte[] fullName) throws IllegalArgumentException {
395    return valueOf(fullName, 0, fullName.length);
396  }
397
398  /**
399   * Construct a TableName
400   * @param fullName byte array to look into
401   * @param offset   within said array
402   * @param length   within said array
403   * @throws IllegalArgumentException if fullName equals old root or old meta.
404   */
405  public static TableName valueOf(byte[] fullName, int offset, int length)
406    throws IllegalArgumentException {
407    Preconditions.checkArgument(offset >= 0, "offset must be non-negative but was %s", offset);
408    Preconditions.checkArgument(offset < fullName.length, "offset (%s) must be < array length (%s)",
409      offset, fullName.length);
410    Preconditions.checkArgument(length <= fullName.length,
411      "length (%s) must be <= array length (%s)", length, fullName.length);
412    for (TableName tn : tableCache) {
413      final byte[] tnName = tn.getName();
414      if (Bytes.equals(tnName, 0, tnName.length, fullName, offset, length)) {
415        return tn;
416      }
417    }
418
419    int namespaceDelimIndex = ArrayUtils.lastIndexOf(fullName, (byte) NAMESPACE_DELIM);
420
421    if (namespaceDelimIndex < offset) {
422      return createTableNameIfNecessary(ByteBuffer.wrap(NamespaceDescriptor.DEFAULT_NAMESPACE_NAME),
423        ByteBuffer.wrap(fullName, offset, length));
424    } else {
425      return createTableNameIfNecessary(ByteBuffer.wrap(fullName, offset, namespaceDelimIndex),
426        ByteBuffer.wrap(fullName, namespaceDelimIndex + 1, length - (namespaceDelimIndex + 1)));
427    }
428  }
429
430  /**
431   * Construct a TableName
432   * @param fullname of a table, possibly with a leading namespace and ':' as delimiter.
433   * @throws IllegalArgumentException if fullName equals old root or old meta.
434   */
435  public static TableName valueOf(ByteBuffer fullname) {
436    fullname = fullname.duplicate();
437    fullname.mark();
438    boolean miss = true;
439    while (fullname.hasRemaining() && miss) {
440      miss = ((byte) NAMESPACE_DELIM) != fullname.get();
441    }
442    if (miss) {
443      fullname.reset();
444      return valueOf(null, fullname);
445    } else {
446      ByteBuffer qualifier = fullname.slice();
447      int delimiterIndex = fullname.position() - 1;
448      fullname.reset();
449      // changing variable name for clarity
450      ByteBuffer namespace = fullname.duplicate();
451      namespace.limit(delimiterIndex);
452      return valueOf(namespace, qualifier);
453    }
454  }
455
456  /**
457   * Construct a TableName
458   * @throws IllegalArgumentException if fullName equals old root or old meta. Some code depends on
459   *                                  this.
460   */
461  public static TableName valueOf(String name) {
462    for (TableName tn : tableCache) {
463      if (name.equals(tn.getNameAsString())) {
464        return tn;
465      }
466    }
467
468    final int namespaceDelimIndex = name.indexOf(NAMESPACE_DELIM);
469
470    if (namespaceDelimIndex < 0) {
471      return createTableNameIfNecessary(ByteBuffer.wrap(NamespaceDescriptor.DEFAULT_NAMESPACE_NAME),
472        ByteBuffer.wrap(Bytes.toBytes(name)));
473    } else {
474      // indexOf is by character, not byte (consider multi-byte characters)
475      String ns = name.substring(0, namespaceDelimIndex);
476      String qualifier = name.substring(namespaceDelimIndex + 1);
477      return createTableNameIfNecessary(ByteBuffer.wrap(Bytes.toBytes(ns)),
478        ByteBuffer.wrap(Bytes.toBytes(qualifier)));
479    }
480  }
481
482  public static TableName valueOf(byte[] namespace, byte[] qualifier) {
483    if (namespace == null || namespace.length < 1) {
484      namespace = NamespaceDescriptor.DEFAULT_NAMESPACE_NAME;
485    }
486
487    for (TableName tn : tableCache) {
488      if (
489        Arrays.equals(tn.getQualifier(), qualifier) && Arrays.equals(tn.getNamespace(), namespace)
490      ) {
491        return tn;
492      }
493    }
494
495    return createTableNameIfNecessary(ByteBuffer.wrap(namespace), ByteBuffer.wrap(qualifier));
496  }
497
498  public static TableName valueOf(ByteBuffer namespace, ByteBuffer qualifier) {
499    if (namespace == null || namespace.remaining() < 1) {
500      return createTableNameIfNecessary(ByteBuffer.wrap(NamespaceDescriptor.DEFAULT_NAMESPACE_NAME),
501        qualifier);
502    }
503
504    return createTableNameIfNecessary(namespace, qualifier);
505  }
506
507  @Override
508  public boolean equals(Object o) {
509    if (this == o) return true;
510    if (o == null || getClass() != o.getClass()) return false;
511
512    TableName tableName = (TableName) o;
513
514    return o.hashCode() == hashCode && nameAsString.equals(tableName.nameAsString);
515  }
516
517  @Override
518  public int hashCode() {
519    return hashCode;
520  }
521
522  @Override
523  public int compareTo(TableName tableName) {
524    // For performance reasons, the ordering is not lexicographic.
525    if (this == tableName) {
526      return 0;
527    }
528    if (this.hashCode < tableName.hashCode()) {
529      return -1;
530    }
531    if (this.hashCode > tableName.hashCode()) {
532      return 1;
533    }
534    return this.nameAsString.compareTo(tableName.getNameAsString());
535  }
536
537}