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