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.codec.prefixtree.encode.tokenize;
20  
21  import java.util.ArrayList;
22  import java.util.List;
23  
24  import org.apache.hadoop.hbase.classification.InterfaceAudience;
25  import org.apache.hadoop.hbase.util.ByteRange;
26  import org.apache.hadoop.hbase.util.ByteRangeUtils;
27  import org.apache.hadoop.hbase.util.Bytes;
28  import org.apache.hadoop.hbase.util.CollectionUtils;
29  import org.apache.hadoop.hbase.util.SimpleMutableByteRange;
30  import org.apache.hadoop.hbase.util.Strings;
31  
32  import com.google.common.collect.Lists;
33  
34  /**
35   * Individual node in a Trie structure.  Each node is one of 3 types:
36   * <li>Branch: an internal trie node that may have a token and must have multiple children, but does
37   * not represent an actual input byte[], hence its numOccurrences is 0
38   * <li>Leaf: a node with no children and where numOccurrences is >= 1.  It's token represents the
39   * last bytes in the input byte[]s.
40   * <li>Nub: a combination of a branch and leaf.  Its token represents the last bytes of input
41   * byte[]s and has numOccurrences >= 1, but it also has child nodes which represent input byte[]s
42   * that add bytes to this nodes input byte[].
43   * <br/><br/>
44   * Example inputs (numInputs=7):
45   * 0: AAA
46   * 1: AAA
47   * 2: AAB
48   * 3: AAB
49   * 4: AAB
50   * 5: AABQQ
51   * 6: AABQQ
52   * <br/><br/>
53   * Resulting TokenizerNodes:
54   * AA <- branch, numOccurrences=0, tokenStartOffset=0, token.length=2
55   * A  <- leaf, numOccurrences=2, tokenStartOffset=2, token.length=1
56   * B  <- nub, numOccurrences=3, tokenStartOffset=2, token.length=1
57   * QQ <- leaf, numOccurrences=2, tokenStartOffset=3, token.length=2
58   * <br/><br/>
59   * numInputs == 7 == sum(numOccurrences) == 0 + 2 + 3 + 2
60   */
61  @InterfaceAudience.Private
62  public class TokenizerNode{
63  
64    /*
65     * Ref to data structure wrapper
66     */
67    protected Tokenizer builder;
68  
69    /****************************************************************** 
70     * Tree content/structure used during tokenization 
71     * ****************************************************************/
72  
73    /*
74     * ref to parent trie node
75     */
76    protected TokenizerNode parent;
77  
78    /*
79     * node depth in trie, irrespective of each node's token length
80     */
81    protected int nodeDepth;
82  
83    /*
84     * start index of this token in original byte[]
85     */
86    protected int tokenStartOffset;
87  
88    /*
89     * bytes for this trie node.  can be length 0 in root node
90     */
91    protected ByteRange token;
92  
93    /*
94     * A count of occurrences in the input byte[]s, not the trie structure. 0 for branch nodes, 1+ for
95     * nubs and leaves. If the same byte[] is added to the trie multiple times, this is the only thing
96     * that changes in the tokenizer. As a result, duplicate byte[]s are very inexpensive to encode.
97     */
98    protected int numOccurrences;
99  
100   /*
101    * The maximum fan-out of a byte[] trie is 256, so there are a maximum of 256
102    * child nodes.
103    */
104   protected ArrayList<TokenizerNode> children;
105 
106 
107   /*
108    * Fields used later in the encoding process for sorting the nodes into the order they'll be
109    * written to the output byte[].  With these fields, the TokenizerNode and therefore Tokenizer
110    * are not generic data structures but instead are specific to HBase PrefixTree encoding. 
111    */
112 
113   /*
114    * unique id assigned to each TokenizerNode
115    */
116   protected long id;
117 
118   /*
119    * set >=0 for nubs and leaves
120    */
121   protected int firstInsertionIndex = -1;
122 
123   /*
124    * A positive value indicating how many bytes before the end of the block this node will start. If
125    * the section is 55 bytes and negativeOffset is 9, then the node will start at 46.
126    */
127   protected int negativeIndex = 0;
128 
129   /*
130    * The offset in the output array at which to start writing this node's token bytes.  Influenced
131    * by the lengths of all tokens sorted before this one.
132    */
133   protected int outputArrayOffset = -1;
134 
135 
136   /*********************** construct *****************************/
137 
138   public TokenizerNode(Tokenizer builder, TokenizerNode parent, int nodeDepth,
139       int tokenStartOffset, int tokenOffset, int tokenLength) {
140     this.token = new SimpleMutableByteRange();
141     reconstruct(builder, parent, nodeDepth, tokenStartOffset, tokenOffset, tokenLength);
142     this.children = Lists.newArrayList();
143   }
144 
145   /*
146    * Sub-constructor for initializing all fields without allocating a new object.  Used by the
147    * regular constructor.
148    */
149   public void reconstruct(Tokenizer builder, TokenizerNode parent, int nodeDepth,
150       int tokenStartOffset, int tokenOffset, int tokenLength) {
151     this.builder = builder;
152     this.id = builder.nextNodeId();
153     this.parent = parent;
154     this.nodeDepth = nodeDepth;
155     builder.submitMaxNodeDepthCandidate(nodeDepth);
156     this.tokenStartOffset = tokenStartOffset;
157     this.token.set(builder.tokens, tokenOffset, tokenLength);
158     this.numOccurrences = 1;
159   }
160 
161   /*
162    * Clear the state of this node so that it looks like it was just allocated.
163    */
164   public void reset() {
165     builder = null;
166     parent = null;
167     nodeDepth = 0;
168     tokenStartOffset = 0;
169     token.unset();
170     numOccurrences = 0;
171     children.clear();// branches & nubs
172 
173     // ids/offsets. used during writing to byte[]
174     id = 0;
175     firstInsertionIndex = -1;// set >=0 for nubs and leaves
176     negativeIndex = 0;
177     outputArrayOffset = -1;
178   }
179 
180 
181   /************************* building *********************************/
182 
183   /*
184    * <li>Only public method used during the tokenization process
185    * <li>Requires that the input ByteRange sort after the previous, and therefore after all previous
186    * inputs
187    * <li>Only looks at bytes of the input array that align with this node's token
188    */
189   public void addSorted(final ByteRange bytes) {// recursively build the tree
190 
191     /*
192      * Recurse deeper into the existing trie structure
193      */
194     if (matchesToken(bytes) && CollectionUtils.notEmpty(children)) {
195       TokenizerNode lastChild = CollectionUtils.getLast(children);
196       if (lastChild.partiallyMatchesToken(bytes)) {
197         lastChild.addSorted(bytes);
198         return;
199       }
200     }
201 
202     /*
203      * Recursion ended.  We must either
204      * <li>1: increment numOccurrences if this input was equal to the previous
205      * <li>2: convert this node from a leaf to a nub, and add a new child leaf
206      * <li>3: split this node into a branch and leaf, and then add a second leaf
207      */
208 
209     // add it as a child of this node
210     int numIdenticalTokenBytes = numIdenticalBytes(bytes);// should be <= token.length
211     int tailOffset = tokenStartOffset + numIdenticalTokenBytes;
212     int tailLength = bytes.getLength() - tailOffset;
213 
214     if (numIdenticalTokenBytes == token.getLength()) {
215       if (tailLength == 0) {// identical to this node (case 1)
216         incrementNumOccurrences(1);
217       } else {// identical to this node, but with a few extra tailing bytes. (leaf -> nub) (case 2)
218         int childNodeDepth = nodeDepth + 1;
219         int childTokenStartOffset = tokenStartOffset + numIdenticalTokenBytes;
220         TokenizerNode newChildNode = builder.addNode(this, childNodeDepth, childTokenStartOffset,
221           bytes, tailOffset);
222         addChild(newChildNode);
223       }
224     } else {//numIdenticalBytes > 0, split into branch/leaf and then add second leaf (case 3)
225       split(numIdenticalTokenBytes, bytes);
226     }
227   }
228 
229 
230   protected void addChild(TokenizerNode node) {
231     node.setParent(this);
232     children.add(node);
233   }
234 
235 
236   /**
237    * Called when we need to convert a leaf node into a branch with 2 leaves. Comments inside the
238    * method assume we have token BAA starting at tokenStartOffset=0 and are adding BOO. The output
239    * will be 3 nodes:<br/>
240    * <li>1: B <- branch
241    * <li>2: AA <- leaf
242    * <li>3: OO <- leaf
243    *
244    * @param numTokenBytesToRetain => 1 (the B)
245    * @param bytes => BOO
246    */
247   protected void split(int numTokenBytesToRetain, final ByteRange bytes) {
248     int childNodeDepth = nodeDepth;
249     int childTokenStartOffset = tokenStartOffset + numTokenBytesToRetain;
250 
251     //create leaf AA
252     TokenizerNode firstChild = builder.addNode(this, childNodeDepth, childTokenStartOffset,
253       token, numTokenBytesToRetain);
254     firstChild.setNumOccurrences(numOccurrences);// do before clearing this node's numOccurrences
255     token.setLength(numTokenBytesToRetain);//shorten current token from BAA to B
256     numOccurrences = 0;//current node is now a branch
257 
258     moveChildrenToDifferentParent(firstChild);//point the new leaf (AA) to the new branch (B)
259     addChild(firstChild);//add the new leaf (AA) to the branch's (B's) children
260 
261     //create leaf OO
262     TokenizerNode secondChild = builder.addNode(this, childNodeDepth, childTokenStartOffset,
263       bytes, tokenStartOffset + numTokenBytesToRetain);
264     addChild(secondChild);//add the new leaf (00) to the branch's (B's) children
265 
266     // we inserted branch node B as a new level above/before the two children, so increment the
267     // depths of the children below
268     firstChild.incrementNodeDepthRecursively();
269     secondChild.incrementNodeDepthRecursively();
270   }
271 
272 
273   protected void incrementNodeDepthRecursively() {
274     ++nodeDepth;
275     builder.submitMaxNodeDepthCandidate(nodeDepth);
276     for (int i = 0; i < children.size(); ++i) {
277       children.get(i).incrementNodeDepthRecursively();
278     }
279   }
280 
281 
282   protected void moveChildrenToDifferentParent(TokenizerNode newParent) {
283     for (int i = 0; i < children.size(); ++i) {
284       TokenizerNode child = children.get(i);
285       child.setParent(newParent);
286       newParent.children.add(child);
287     }
288     children.clear();
289   }
290 
291 
292   /************************ byte[] utils *************************/
293 
294   protected boolean partiallyMatchesToken(ByteRange bytes) {
295     return numIdenticalBytes(bytes) > 0;
296   }
297 
298   protected boolean matchesToken(ByteRange bytes) {
299     return numIdenticalBytes(bytes) == getTokenLength();
300   }
301 
302   protected int numIdenticalBytes(ByteRange bytes) {
303     return ByteRangeUtils.numEqualPrefixBytes(token, bytes, tokenStartOffset);
304   }
305 
306 
307   /***************** moving nodes around ************************/
308 
309   public void appendNodesToExternalList(List<TokenizerNode> appendTo, boolean includeNonLeaves,
310       boolean includeLeaves) {
311     if (includeNonLeaves && !isLeaf() || includeLeaves && isLeaf()) {
312       appendTo.add(this);
313     }
314     for (int i = 0; i < children.size(); ++i) {
315       TokenizerNode child = children.get(i);
316       child.appendNodesToExternalList(appendTo, includeNonLeaves, includeLeaves);
317     }
318   }
319 
320   public int setInsertionIndexes(int nextIndex) {
321     int newNextIndex = nextIndex;
322     if (hasOccurrences()) {
323       setFirstInsertionIndex(nextIndex);
324       newNextIndex += numOccurrences;
325     }
326     for (int i = 0; i < children.size(); ++i) {
327       TokenizerNode child = children.get(i);
328       newNextIndex = child.setInsertionIndexes(newNextIndex);
329     }
330     return newNextIndex;
331   }
332 
333   public void appendOutputArrayOffsets(List<Integer> offsets) {
334     if (hasOccurrences()) {
335       offsets.add(outputArrayOffset);
336     }
337     for (int i = 0; i < children.size(); ++i) {
338       TokenizerNode child = children.get(i);
339       child.appendOutputArrayOffsets(offsets);
340     }
341   }
342 
343 
344   /***************** searching *********************************/
345 
346   /*
347    * Do a trie style search through the tokenizer.  One option for looking up families or qualifiers
348    * during encoding, but currently unused in favor of tracking this information as they are added.
349    *
350    * Keeping code pending further performance testing.
351    */
352   public void getNode(TokenizerRowSearchResult resultHolder, byte[] key, int keyOffset,
353       int keyLength) {
354     int thisNodeDepthPlusLength = tokenStartOffset + token.getLength();
355 
356     // quick check if the key is shorter than this node (may not work for binary search)
357     if (CollectionUtils.isEmpty(children)) {
358       if (thisNodeDepthPlusLength < keyLength) {// ran out of bytes
359         resultHolder.set(TokenizerRowSearchPosition.NO_MATCH, null);
360         return;
361       }
362     }
363 
364     // all token bytes must match
365     for (int i = 0; i < token.getLength(); ++i) {
366       if (key[tokenStartOffset + keyOffset + i] != token.get(i)) {
367         // TODO return whether it's before or after so we can binary search
368         resultHolder.set(TokenizerRowSearchPosition.NO_MATCH, null);
369         return;
370       }
371     }
372 
373     if (thisNodeDepthPlusLength == keyLength && numOccurrences > 0) {
374       resultHolder.set(TokenizerRowSearchPosition.MATCH, this);// MATCH
375       return;
376     }
377 
378     if (CollectionUtils.notEmpty(children)) {
379       // TODO binary search the children
380       for (int i = 0; i < children.size(); ++i) {
381         TokenizerNode child = children.get(i);
382         child.getNode(resultHolder, key, keyOffset, keyLength);
383         if (resultHolder.isMatch()) {
384           return;
385         } else if (resultHolder.getDifference() == TokenizerRowSearchPosition.BEFORE) {
386           // passed it, so it doesn't exist
387           resultHolder.set(TokenizerRowSearchPosition.NO_MATCH, null);
388           return;
389         }
390         // key is still AFTER the current node, so continue searching
391       }
392     }
393 
394     // checked all children (or there were no children), and didn't find it
395     resultHolder.set(TokenizerRowSearchPosition.NO_MATCH, null);
396     return;
397   }
398 
399 
400   /****************** writing back to byte[]'s *************************/
401 
402   public byte[] getNewByteArray() {
403     byte[] arrayToFill = new byte[tokenStartOffset + token.getLength()];
404     fillInBytes(arrayToFill);
405     return arrayToFill;
406   }
407 
408   public void fillInBytes(byte[] arrayToFill) {
409     for (int i = 0; i < token.getLength(); ++i) {
410       arrayToFill[tokenStartOffset + i] = token.get(i);
411     }
412     if (parent != null) {
413       parent.fillInBytes(arrayToFill);
414     }
415   }
416 
417 
418   /************************** printing ***********************/
419 
420   @Override
421   public String toString() {
422     String s = "";
423     if (parent == null) {
424       s += "R ";
425     } else {
426       s += getBnlIndicator(false) + " " + Bytes.toString(parent.getNewByteArray());
427     }
428     s += "[" + Bytes.toString(token.deepCopyToNewArray()) + "]";
429     if (numOccurrences > 0) {
430       s += "x" + numOccurrences;
431     }
432     return s;
433   }
434 
435   public String getPaddedTokenAndOccurrenceString() {
436     StringBuilder sb = new StringBuilder();
437     sb.append(getBnlIndicator(true));
438     sb.append(Strings.padFront(numOccurrences + "", ' ', 3));
439     sb.append(Strings.padFront(nodeDepth + "", ' ', 3));
440     if (outputArrayOffset >= 0) {
441       sb.append(Strings.padFront(outputArrayOffset + "", ' ', 3));
442     }
443     sb.append("  ");
444     for (int i = 0; i < tokenStartOffset; ++i) {
445       sb.append(" ");
446     }
447     sb.append(Bytes.toString(token.deepCopyToNewArray()).replaceAll(" ", "_"));
448     return sb.toString();
449   }
450 
451   public String getBnlIndicator(boolean indent) {
452     if (indent) {
453       if (isNub()) {
454         return " N ";
455       }
456       return isBranch() ? "B  " : "  L";
457     }
458     if (isNub()) {
459       return "N";
460     }
461     return isBranch() ? "B" : "L";
462   }
463 
464 
465   /********************** count different node types ********************/
466 
467   public int getNumBranchNodesIncludingThisNode() {
468     if (isLeaf()) {
469       return 0;
470     }
471     int totalFromThisPlusChildren = isBranch() ? 1 : 0;
472     for (int i = 0; i < children.size(); ++i) {
473       TokenizerNode child = children.get(i);
474       totalFromThisPlusChildren += child.getNumBranchNodesIncludingThisNode();
475     }
476     return totalFromThisPlusChildren;
477   }
478 
479   public int getNumNubNodesIncludingThisNode() {
480     if (isLeaf()) {
481       return 0;
482     }
483     int totalFromThisPlusChildren = isNub() ? 1 : 0;
484     for (int i = 0; i < children.size(); ++i) {
485       TokenizerNode child = children.get(i);
486       totalFromThisPlusChildren += child.getNumNubNodesIncludingThisNode();
487     }
488     return totalFromThisPlusChildren;
489   }
490 
491   public int getNumLeafNodesIncludingThisNode() {
492     if (isLeaf()) {
493       return 1;
494     }
495     int totalFromChildren = 0;
496     for (int i = 0; i < children.size(); ++i) {
497       TokenizerNode child = children.get(i);
498       totalFromChildren += child.getNumLeafNodesIncludingThisNode();
499     }
500     return totalFromChildren;
501   }
502 
503 
504   /*********************** simple read-only methods *******************************/
505 
506   public int getNodeDepth() {
507     return nodeDepth;
508   }
509 
510   public int getTokenLength() {
511     return token.getLength();
512   }
513 
514   public boolean hasOccurrences() {
515     return numOccurrences > 0;
516   }
517 
518   public boolean isRoot() {
519     return this.parent == null;
520   }
521 
522   public int getNumChildren() {
523     return CollectionUtils.nullSafeSize(children);
524   }
525 
526   public TokenizerNode getLastChild() {
527     if (CollectionUtils.isEmpty(children)) {
528       return null;
529     }
530     return CollectionUtils.getLast(children);
531   }
532 
533   public boolean isLeaf() {
534     return CollectionUtils.isEmpty(children) && hasOccurrences();
535   }
536 
537   public boolean isBranch() {
538     return CollectionUtils.notEmpty(children) && !hasOccurrences();
539   }
540 
541   public boolean isNub() {
542     return CollectionUtils.notEmpty(children) && hasOccurrences();
543   }
544 
545 
546   /********************** simple mutation methods *************************/
547 
548   /**
549    * Each occurrence > 1 indicates a repeat of the previous entry.  This can be called directly by
550    * an external class without going through the process of detecting a repeat if it is a known
551    * repeat by some external mechanism.  PtEncoder uses this when adding cells to a row if it knows
552    * the new cells are part of the current row.
553    * @param d increment by this amount
554    */
555   public void incrementNumOccurrences(int d) {
556     numOccurrences += d;
557   }
558 
559 
560   /************************* autogenerated get/set ******************/
561 
562   public int getTokenOffset() {
563     return tokenStartOffset;
564   }
565 
566   public TokenizerNode getParent() {
567     return parent;
568   }
569 
570   public ByteRange getToken() {
571     return token;
572   }
573 
574   public int getNumOccurrences() {
575     return numOccurrences;
576   }
577 
578   public void setParent(TokenizerNode parent) {
579     this.parent = parent;
580   }
581 
582   public void setNumOccurrences(int numOccurrences) {
583     this.numOccurrences = numOccurrences;
584   }
585 
586   public ArrayList<TokenizerNode> getChildren() {
587     return children;
588   }
589 
590   public long getId() {
591     return id;
592   }
593 
594   public int getFirstInsertionIndex() {
595     return firstInsertionIndex;
596   }
597 
598   public void setFirstInsertionIndex(int firstInsertionIndex) {
599     this.firstInsertionIndex = firstInsertionIndex;
600   }
601 
602   public int getNegativeIndex() {
603     return negativeIndex;
604   }
605 
606   public void setNegativeIndex(int negativeIndex) {
607     this.negativeIndex = negativeIndex;
608   }
609 
610   public int getOutputArrayOffset() {
611     return outputArrayOffset;
612   }
613 
614   public void setOutputArrayOffset(int outputArrayOffset) {
615     this.outputArrayOffset = outputArrayOffset;
616   }
617 
618   public void setId(long id) {
619     this.id = id;
620   }
621 
622   public void setBuilder(Tokenizer builder) {
623     this.builder = builder;
624   }
625 
626   public void setTokenOffset(int tokenOffset) {
627     this.tokenStartOffset = tokenOffset;
628   }
629 
630   public void setToken(ByteRange token) {
631     this.token = token;
632   }
633 
634 }