View Javadoc

1   /**
2    * Licensed to the Apache Software Foundation (ASF) under one
3    * or more contributor license agreements.  See the NOTICE file
4    * distributed with this work for additional information
5    * regarding copyright ownership.  The ASF licenses this file
6    * to you under the Apache License, Version 2.0 (the
7    * "License"); you may not use this file except in compliance
8    * with the License.  You may obtain a copy of the License at
9    *
10   *     http://www.apache.org/licenses/LICENSE-2.0
11   *
12   * Unless required by applicable law or agreed to in writing, software
13   * distributed under the License is distributed on an "AS IS" BASIS,
14   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15   * See the License for the specific language governing permissions and
16   * limitations under the License.
17   */
18  
19  package org.apache.hadoop.hbase;
20  
21  import org.apache.hadoop.hbase.classification.InterfaceAudience;
22  import org.apache.hadoop.hbase.classification.InterfaceStability;
23  import org.apache.hadoop.hbase.KeyValue.KVComparator;
24  import org.apache.hadoop.hbase.util.Bytes;
25  
26  import java.nio.ByteBuffer;
27  import java.util.Arrays;
28  import java.util.Set;
29  import java.util.concurrent.CopyOnWriteArraySet;
30  
31  /**
32   * Immutable POJO class for representing a table name.
33   * Which is of the form:
34   * <table namespace>:<table qualifier>
35   *
36   * Two special namespaces:
37   *
38   * 1. hbase - system namespace, used to contain hbase internal tables
39   * 2. default - tables with no explicit specified namespace will
40   * automatically fall into this namespace.
41   *
42   * ie
43   *
44   * a) foo:bar, means namespace=foo and qualifier=bar
45   * b) bar, means namespace=default and qualifier=bar
46   * c) default:bar, means namespace=default and qualifier=bar
47   *
48   *  <p>
49   * Internally, in this class, we cache the instances to limit the number of objects and
50   *  make the "equals" faster. We try to minimize the number of objects created of
51   *  the number of array copy to check if we already have an instance of this TableName. The code
52   *  is not optimize for a new instance creation but is optimized to check for existence.
53   * </p>
54   */
55  @InterfaceAudience.Public
56  @InterfaceStability.Evolving
57  public final class TableName implements Comparable<TableName> {
58  
59    /** See {@link #createTableNameIfNecessary(ByteBuffer, ByteBuffer)} */
60    private static final Set<TableName> tableCache = new CopyOnWriteArraySet<TableName>();
61  
62    /** Namespace delimiter */
63    //this should always be only 1 byte long
64    public final static char NAMESPACE_DELIM = ':';
65  
66    // A non-capture group so that this can be embedded.
67    // regex is a bit more complicated to support nuance of tables
68    // in default namespace
69    //Allows only letters, digits and '_'
70    public static final String VALID_NAMESPACE_REGEX =
71        "(?:[a-zA-Z_0-9]+)";
72    //Allows only letters, digits, '_', '-' and '.'
73    public static final String VALID_TABLE_QUALIFIER_REGEX =
74        "(?:[a-zA-Z_0-9][a-zA-Z_0-9-.]*)";
75    //Concatenation of NAMESPACE_REGEX and TABLE_QUALIFIER_REGEX,
76    //with NAMESPACE_DELIM as delimiter
77    public static final String VALID_USER_TABLE_REGEX =
78        "(?:(?:(?:"+VALID_NAMESPACE_REGEX+"\\"+NAMESPACE_DELIM+")?)" +
79           "(?:"+VALID_TABLE_QUALIFIER_REGEX+"))";
80  
81    /** The hbase:meta table's name. */
82    public static final TableName META_TABLE_NAME =
83        valueOf(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR, "meta");
84  
85    /** The Namespace table's name. */
86    public static final TableName NAMESPACE_TABLE_NAME =
87        valueOf(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR, "namespace");
88  
89    public static final String OLD_META_STR = ".META.";
90    public static final String OLD_ROOT_STR = "-ROOT-";
91  
92  
93  
94    /**
95     * TableName for old -ROOT- table. It is used to read/process old WALs which have
96     * ROOT edits.
97     */
98    public static final TableName OLD_ROOT_TABLE_NAME = getADummyTableName(OLD_ROOT_STR);
99    /**
100    * TableName for old .META. table. Used in testing.
101    */
102   public static final TableName OLD_META_TABLE_NAME = getADummyTableName(OLD_META_STR);
103 
104   private final byte[] name;
105   private final String nameAsString;
106   private final byte[] namespace;
107   private final String namespaceAsString;
108   private final byte[] qualifier;
109   private final String qualifierAsString;
110   private final boolean systemTable;
111   private final int hashCode;
112 
113   /**
114    * Check passed byte array, "tableName", is legal user-space table name.
115    * @return Returns passed <code>tableName</code> param
116    * @throws IllegalArgumentException if passed a tableName is null or
117    * is made of other than 'word' characters or underscores: i.e.
118    * <code>[a-zA-Z_0-9.-:]</code>. The ':' is used to delimit the namespace
119    * from the table name and can be used for nothing else.
120    *
121    * Namespace names can only contain 'word' characters
122    * <code>[a-zA-Z_0-9]</code> or '_'
123    *
124    * Qualifier names can only contain 'word' characters
125    * <code>[a-zA-Z_0-9]</code> or '_', '.' or '-'.
126    * The name may not start with '.' or '-'.
127    *
128    * Valid fully qualified table names:
129    * foo:bar, namespace=&gt;foo, table=&gt;bar
130    * org:foo.bar, namespace=org, table=&gt;foo.bar
131    */
132   public static byte [] isLegalFullyQualifiedTableName(final byte[] tableName) {
133     if (tableName == null || tableName.length <= 0) {
134       throw new IllegalArgumentException("Name is null or empty");
135     }
136 
137     int namespaceDelimIndex = com.google.common.primitives.Bytes.lastIndexOf(tableName,
138         (byte) NAMESPACE_DELIM);
139     if (namespaceDelimIndex < 0){
140       isLegalTableQualifierName(tableName);
141     } else {
142       isLegalNamespaceName(tableName, 0, namespaceDelimIndex);
143       isLegalTableQualifierName(tableName, namespaceDelimIndex + 1, tableName.length);
144     }
145     return tableName;
146   }
147 
148   public static byte [] isLegalTableQualifierName(final byte[] qualifierName) {
149     isLegalTableQualifierName(qualifierName, 0, qualifierName.length, false);
150     return qualifierName;
151   }
152 
153   public static byte [] isLegalTableQualifierName(final byte[] qualifierName, boolean isSnapshot) {
154     isLegalTableQualifierName(qualifierName, 0, qualifierName.length, isSnapshot);
155     return qualifierName;
156   }
157 
158 
159   /**
160    * Qualifier names can only contain 'word' characters
161    * <code>[a-zA-Z_0-9]</code> or '_', '.' or '-'.
162    * The name may not start with '.' or '-'.
163    *
164    * @param qualifierName byte array containing the qualifier name
165    * @param start start index
166    * @param end end index (exclusive)
167    */
168   public static void isLegalTableQualifierName(final byte[] qualifierName,
169                                                 int start,
170                                                 int end) {
171       isLegalTableQualifierName(qualifierName, start, end, false);
172   }
173 
174   public static void isLegalTableQualifierName(final byte[] qualifierName,
175                                                 int start,
176                                                 int end,
177                                                 boolean isSnapshot) {
178     if(end - start < 1) {
179       throw new IllegalArgumentException(isSnapshot ? "Snapshot" : "Table" + " qualifier must not be empty");
180     }
181 
182     if (qualifierName[start] == '.' || qualifierName[start] == '-') {
183       throw new IllegalArgumentException("Illegal first character <" + qualifierName[start] +
184                                          "> at 0. " + (isSnapshot ? "Snapshot" : "User-space table") +
185                                          " qualifiers can only start with 'alphanumeric " +
186                                          "characters': i.e. [a-zA-Z_0-9]: " +
187                                          Bytes.toString(qualifierName, start, end));
188     }
189     for (int i = start; i < end; i++) {
190       if (Character.isLetterOrDigit(qualifierName[i]) ||
191           qualifierName[i] == '_' ||
192           qualifierName[i] == '-' ||
193           qualifierName[i] == '.') {
194         continue;
195       }
196       throw new IllegalArgumentException("Illegal character code:" + qualifierName[i] +
197                                          ", <" + (char) qualifierName[i] + "> at " + i +
198                                          ". " + (isSnapshot ? "Snapshot" : "User-space table") +
199                                          " qualifiers can only contain " +
200                                          "'alphanumeric characters': i.e. [a-zA-Z_0-9-.]: " +
201                                          Bytes.toString(qualifierName, start, end));
202     }
203   }
204   public static void isLegalNamespaceName(byte[] namespaceName) {
205     isLegalNamespaceName(namespaceName, 0, namespaceName.length);
206   }
207 
208   /**
209    * Valid namespace characters are [a-zA-Z_0-9]
210    */
211   public static void isLegalNamespaceName(final byte[] namespaceName,
212                                            final int start,
213                                            final int end) {
214     if(end - start < 1) {
215       throw new IllegalArgumentException("Namespace name must not be empty");
216     }
217     for (int i = start; i < end; i++) {
218       if (Character.isLetterOrDigit(namespaceName[i])|| namespaceName[i] == '_') {
219         continue;
220       }
221       throw new IllegalArgumentException("Illegal character <" + namespaceName[i] +
222         "> at " + i + ". Namespaces can only contain " +
223         "'alphanumeric characters': i.e. [a-zA-Z_0-9]: " + Bytes.toString(namespaceName,
224           start, end));
225     }
226   }
227 
228   public byte[] getName() {
229     return name;
230   }
231 
232   public String getNameAsString() {
233     return nameAsString;
234   }
235 
236   public byte[] getNamespace() {
237     return namespace;
238   }
239 
240   public String getNamespaceAsString() {
241     return namespaceAsString;
242   }
243 
244   /**
245    * Ideally, getNameAsString should contain namespace within it,
246    * but if the namespace is default, it just returns the name. This method
247    * takes care of this corner case.
248    */
249   public String getNameWithNamespaceInclAsString() {
250     if(getNamespaceAsString().equals(NamespaceDescriptor.DEFAULT_NAMESPACE_NAME_STR)) {
251       return NamespaceDescriptor.DEFAULT_NAMESPACE_NAME_STR +
252           TableName.NAMESPACE_DELIM + getNameAsString();
253     }
254     return getNameAsString();
255   }
256 
257   public byte[] getQualifier() {
258     return qualifier;
259   }
260 
261   public String getQualifierAsString() {
262     return qualifierAsString;
263   }
264 
265   public byte[] toBytes() {
266     return name;
267   }
268 
269   public boolean isSystemTable() {
270     return systemTable;
271   }
272 
273   @Override
274   public String toString() {
275     return nameAsString;
276   }
277 
278   /**
279    *
280    * @throws IllegalArgumentException See {@link #valueOf(byte[])}
281    */
282   private TableName(ByteBuffer namespace, ByteBuffer qualifier) throws IllegalArgumentException {
283     this.qualifier = new byte[qualifier.remaining()];
284     qualifier.duplicate().get(this.qualifier);
285     this.qualifierAsString = Bytes.toString(this.qualifier);
286 
287     if (qualifierAsString.equals(OLD_ROOT_STR)) {
288       throw new IllegalArgumentException(OLD_ROOT_STR + " has been deprecated.");
289     }
290     if (qualifierAsString.equals(OLD_META_STR)) {
291       throw new IllegalArgumentException(OLD_META_STR + " no longer exists. The table has been " +
292           "renamed to " + META_TABLE_NAME);
293     }
294 
295     if (Bytes.equals(NamespaceDescriptor.DEFAULT_NAMESPACE_NAME, namespace)) {
296       // Using the same objects: this will make the comparison faster later
297       this.namespace = NamespaceDescriptor.DEFAULT_NAMESPACE_NAME;
298       this.namespaceAsString = NamespaceDescriptor.DEFAULT_NAMESPACE_NAME_STR;
299       this.systemTable = false;
300 
301       // The name does not include the namespace when it's the default one.
302       this.nameAsString = qualifierAsString;
303       this.name = this.qualifier;
304     } else {
305       if (Bytes.equals(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME, namespace)) {
306         this.namespace = NamespaceDescriptor.SYSTEM_NAMESPACE_NAME;
307         this.namespaceAsString = NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR;
308         this.systemTable = true;
309       } else {
310         this.namespace = new byte[namespace.remaining()];
311         namespace.duplicate().get(this.namespace);
312         this.namespaceAsString = Bytes.toString(this.namespace);
313         this.systemTable = false;
314       }
315       this.nameAsString = namespaceAsString + NAMESPACE_DELIM + qualifierAsString;
316       this.name = Bytes.toBytes(nameAsString);
317     }
318 
319     this.hashCode = nameAsString.hashCode();
320 
321     isLegalNamespaceName(this.namespace);
322     isLegalTableQualifierName(this.qualifier);
323   }
324 
325   /**
326    * This is only for the old and meta tables.
327    */
328   private TableName(String qualifier) {
329     this.qualifier = Bytes.toBytes(qualifier);
330     this.qualifierAsString = qualifier;
331 
332     this.namespace = NamespaceDescriptor.SYSTEM_NAMESPACE_NAME;
333     this.namespaceAsString = NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR;
334     this.systemTable = true;
335 
336     // WARNING: nameAsString is different than name for old meta & root!
337     // This is by design.
338     this.nameAsString = namespaceAsString + NAMESPACE_DELIM + qualifierAsString;
339     this.name = this.qualifier;
340 
341     this.hashCode = nameAsString.hashCode();
342   }
343 
344 
345   /**
346    * Check that the object does not exist already. There are two reasons for creating the objects
347    * only once:
348    * 1) With 100K regions, the table names take ~20MB.
349    * 2) Equals becomes much faster as it's resolved with a reference and an int comparison.
350    */
351   private static TableName createTableNameIfNecessary(ByteBuffer bns, ByteBuffer qns) {
352     for (TableName tn : tableCache) {
353       if (Bytes.equals(tn.getQualifier(), qns) && Bytes.equals(tn.getNamespace(), bns)) {
354         return tn;
355       }
356     }
357 
358     TableName newTable = new TableName(bns, qns);
359     if (tableCache.add(newTable)) {  // Adds the specified element if it is not already present
360       return newTable;
361     }
362 
363     // Someone else added it. Let's find it.
364     for (TableName tn : tableCache) {
365       if (Bytes.equals(tn.getQualifier(), qns) && Bytes.equals(tn.getNamespace(), bns)) {
366         return tn;
367       }
368     }
369     // this should never happen.
370     throw new IllegalStateException(newTable + " was supposed to be in the cache");
371   }
372 
373 
374   /**
375    * It is used to create table names for old META, and ROOT table.
376    * These tables are not really legal tables. They are not added into the cache.
377    * @return a dummy TableName instance (with no validation) for the passed qualifier
378    */
379   private static TableName getADummyTableName(String qualifier) {
380     return new TableName(qualifier);
381   }
382 
383 
384   public static TableName valueOf(String namespaceAsString, String qualifierAsString) {
385     if (namespaceAsString == null || namespaceAsString.length() < 1) {
386       namespaceAsString = NamespaceDescriptor.DEFAULT_NAMESPACE_NAME_STR;
387     }
388 
389     for (TableName tn : tableCache) {
390       if (qualifierAsString.equals(tn.getQualifierAsString()) &&
391           namespaceAsString.equals(tn.getNamespaceAsString())) {
392         return tn;
393       }
394     }
395 
396     return createTableNameIfNecessary(
397         ByteBuffer.wrap(Bytes.toBytes(namespaceAsString)),
398         ByteBuffer.wrap(Bytes.toBytes(qualifierAsString)));
399   }
400 
401 
402   /**
403    * @throws IllegalArgumentException if fullName equals old root or old meta. Some code
404    *  depends on this. The test is buried in the table creation to save on array comparison
405    *  when we're creating a standard table object that will be in the cache.
406    */
407   public static TableName valueOf(byte[] fullName) throws IllegalArgumentException{
408     for (TableName tn : tableCache) {
409       if (Arrays.equals(tn.getName(), fullName)) {
410         return tn;
411       }
412     }
413 
414     int namespaceDelimIndex = com.google.common.primitives.Bytes.lastIndexOf(fullName,
415         (byte) NAMESPACE_DELIM);
416 
417     if (namespaceDelimIndex < 0) {
418       return createTableNameIfNecessary(
419           ByteBuffer.wrap(NamespaceDescriptor.DEFAULT_NAMESPACE_NAME),
420           ByteBuffer.wrap(fullName));
421     } else {
422       return createTableNameIfNecessary(
423           ByteBuffer.wrap(fullName, 0, namespaceDelimIndex),
424           ByteBuffer.wrap(fullName, namespaceDelimIndex + 1,
425               fullName.length - (namespaceDelimIndex + 1)));
426     }
427   }
428 
429 
430   /**
431    * @throws IllegalArgumentException if fullName equals old root or old meta. Some code
432    *  depends on this.
433    */
434   public static TableName valueOf(String name) {
435     for (TableName tn : tableCache) {
436       if (name.equals(tn.getNameAsString())) {
437         return tn;
438       }
439     }
440 
441     int namespaceDelimIndex = name.indexOf(NAMESPACE_DELIM);
442     byte[] nameB = Bytes.toBytes(name);
443 
444     if (namespaceDelimIndex < 0) {
445       return createTableNameIfNecessary(
446           ByteBuffer.wrap(NamespaceDescriptor.DEFAULT_NAMESPACE_NAME),
447           ByteBuffer.wrap(nameB));
448     } else {
449       return createTableNameIfNecessary(
450           ByteBuffer.wrap(nameB, 0, namespaceDelimIndex),
451           ByteBuffer.wrap(nameB, namespaceDelimIndex + 1,
452               nameB.length - (namespaceDelimIndex + 1)));
453     }
454   }
455 
456 
457   public static TableName valueOf(byte[] namespace, byte[] qualifier) {
458     if (namespace == null || namespace.length < 1) {
459       namespace = NamespaceDescriptor.DEFAULT_NAMESPACE_NAME;
460     }
461 
462     for (TableName tn : tableCache) {
463       if (Arrays.equals(tn.getQualifier(), qualifier) &&
464           Arrays.equals(tn.getNamespace(), namespace)) {
465         return tn;
466       }
467     }
468 
469     return createTableNameIfNecessary(
470         ByteBuffer.wrap(namespace), ByteBuffer.wrap(qualifier));
471   }
472 
473   public static TableName valueOf(ByteBuffer namespace, ByteBuffer qualifier) {
474     if (namespace == null || namespace.remaining() < 1) {
475       return createTableNameIfNecessary(
476           ByteBuffer.wrap(NamespaceDescriptor.DEFAULT_NAMESPACE_NAME), qualifier);
477     }
478 
479     return createTableNameIfNecessary(namespace, qualifier);
480   }
481 
482   @Override
483   public boolean equals(Object o) {
484     if (this == o) return true;
485     if (o == null || getClass() != o.getClass()) return false;
486 
487     TableName tableName = (TableName) o;
488 
489     return o.hashCode() == hashCode && nameAsString.equals(tableName.nameAsString);
490   }
491 
492   @Override
493   public int hashCode() {
494     return hashCode;
495   }
496 
497   /**
498    * For performance reasons, the ordering is not lexicographic.
499    */
500   @Override
501   public int compareTo(TableName tableName) {
502     if (this == tableName) return 0;
503     if (this.hashCode < tableName.hashCode()) {
504       return -1;
505     }
506     if (this.hashCode > tableName.hashCode()) {
507       return 1;
508     }
509     return this.nameAsString.compareTo(tableName.getNameAsString());
510   }
511 
512   /**
513    * Get the appropriate row comparator for this table.
514    *
515    * @return The comparator.
516    * @deprecated The comparator is an internal property of the table. Should
517    * not have been exposed here
518    */
519   @InterfaceAudience.Private
520   @Deprecated
521   public KVComparator getRowComparator() {
522      if(TableName.META_TABLE_NAME.equals(this)) {
523       return KeyValue.META_COMPARATOR;
524     }
525     return KeyValue.COMPARATOR;
526   }
527 }