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.ArrayUtils;
26  import org.apache.hadoop.hbase.util.ByteRange;
27  import org.apache.hadoop.hbase.util.Bytes;
28  import org.apache.hadoop.hbase.util.CollectionUtils;
29  
30  import com.google.common.collect.Lists;
31  
32  /**
33   * Data structure used in the first stage of PrefixTree encoding:
34   * <li>accepts a sorted stream of ByteRanges
35   * <li>splits them into a set of tokens, each held by a {@link TokenizerNode}
36   * <li>connects the TokenizerNodes via standard java references
37   * <li>keeps a pool of TokenizerNodes and a reusable byte[] for holding all token content
38   * <p><br>
39   * Mainly used for turning Cell rowKeys into a trie, but also used for family and qualifier
40   * encoding.
41   */
42  @InterfaceAudience.Private
43  public class Tokenizer{
44  
45    /***************** fields **************************/
46  
47    protected int numArraysAdded = 0;
48    protected long lastNodeId = -1;
49    protected ArrayList<TokenizerNode> nodes;
50    protected int numNodes;
51    protected TokenizerNode root;
52    protected byte[] tokens;
53    protected int tokensLength;
54  
55    protected int maxElementLength = 0;
56    // number of levels in the tree assuming root level is 0
57    protected int treeDepth = 0;
58  
59  
60    /******************* construct *******************/
61  
62    public Tokenizer() {
63      this.nodes = Lists.newArrayList();
64      this.tokens = new byte[0];
65    }
66  
67    public void reset() {
68      numArraysAdded = 0;
69      lastNodeId = -1;
70      numNodes = 0;
71      tokensLength = 0;
72      root = null;
73      maxElementLength = 0;
74      treeDepth = 0;
75    }
76  
77  
78    /***************** building *************************/
79  
80    public void addAll(ArrayList<ByteRange> sortedByteRanges) {
81      for (int i = 0; i < sortedByteRanges.size(); ++i) {
82        ByteRange byteRange = sortedByteRanges.get(i);
83        addSorted(byteRange);
84      }
85    }
86  
87    public void addSorted(final ByteRange bytes) {
88      ++numArraysAdded;
89      if (bytes.getLength() > maxElementLength) {
90        maxElementLength = bytes.getLength();
91      }
92      if (root == null) {
93        // nodeDepth of firstNode (non-root) is 1
94        root = addNode(null, 1, 0, bytes, 0);
95      } else {
96        root.addSorted(bytes);
97      }
98    }
99  
100   public void incrementNumOccurrencesOfLatestValue(){
101     CollectionUtils.getLast(nodes).incrementNumOccurrences(1);
102   }
103 
104   protected long nextNodeId() {
105     return ++lastNodeId;
106   }
107 
108   protected TokenizerNode addNode(TokenizerNode parent, int nodeDepth, int tokenStartOffset,
109       final ByteRange token, int inputTokenOffset) {
110     int inputTokenLength = token.getLength() - inputTokenOffset;
111     int tokenOffset = appendTokenAndRepointByteRange(token, inputTokenOffset);
112     TokenizerNode node = null;
113     if (nodes.size() <= numNodes) {
114       node = new TokenizerNode(this, parent, nodeDepth, tokenStartOffset, tokenOffset,
115           inputTokenLength);
116       nodes.add(node);
117     } else {
118       node = nodes.get(numNodes);
119       node.reset();
120       node.reconstruct(this, parent, nodeDepth, tokenStartOffset, tokenOffset, inputTokenLength);
121     }
122     ++numNodes;
123     return node;
124   }
125 
126   protected int appendTokenAndRepointByteRange(final ByteRange token, int inputTokenOffset) {
127     int newOffset = tokensLength;
128     int inputTokenLength = token.getLength() - inputTokenOffset;
129     int newMinimum = tokensLength + inputTokenLength;
130     tokens = ArrayUtils.growIfNecessary(tokens, newMinimum, 2 * newMinimum);
131     token.deepCopySubRangeTo(inputTokenOffset, inputTokenLength, tokens, tokensLength);
132     tokensLength += inputTokenLength;
133     return newOffset;
134   }
135 
136   protected void submitMaxNodeDepthCandidate(int nodeDepth) {
137     if (nodeDepth > treeDepth) {
138       treeDepth = nodeDepth;
139     }
140   }
141 
142 
143   /********************* read ********************/
144 
145   public int getNumAdded(){
146     return numArraysAdded;
147   }
148 
149   // for debugging
150   public ArrayList<TokenizerNode> getNodes(boolean includeNonLeaves, boolean includeLeaves) {
151     ArrayList<TokenizerNode> nodes = Lists.newArrayList();
152     root.appendNodesToExternalList(nodes, includeNonLeaves, includeLeaves);
153     return nodes;
154   }
155 
156   public void appendNodes(List<TokenizerNode> appendTo, boolean includeNonLeaves,
157       boolean includeLeaves) {
158     root.appendNodesToExternalList(appendTo, includeNonLeaves, includeLeaves);
159   }
160 
161   public List<byte[]> getArrays() {
162     List<TokenizerNode> nodes = new ArrayList<TokenizerNode>();
163     root.appendNodesToExternalList(nodes, true, true);
164     List<byte[]> byteArrays = Lists.newArrayListWithCapacity(CollectionUtils.nullSafeSize(nodes));
165     for (int i = 0; i < nodes.size(); ++i) {
166       TokenizerNode node = nodes.get(i);
167       for (int j = 0; j < node.getNumOccurrences(); ++j) {
168         byte[] byteArray = node.getNewByteArray();
169         byteArrays.add(byteArray);
170       }
171     }
172     return byteArrays;
173   }
174 
175   //currently unused, but working and possibly useful in the future
176   public void getNode(TokenizerRowSearchResult resultHolder, byte[] key, int keyOffset,
177       int keyLength) {
178     root.getNode(resultHolder, key, keyOffset, keyLength);
179   }
180 
181 
182   /********************** write ***************************/
183 
184   public Tokenizer setNodeFirstInsertionIndexes() {
185     root.setInsertionIndexes(0);
186     return this;
187   }
188 
189   public Tokenizer appendOutputArrayOffsets(List<Integer> offsets) {
190     root.appendOutputArrayOffsets(offsets);
191     return this;
192   }
193 
194 
195   /********************* print/debug ********************/
196 
197   protected static final Boolean INCLUDE_FULL_TREE_IN_TO_STRING = false;
198 
199   @Override
200   public String toString() {
201     StringBuilder sb = new StringBuilder();
202     sb.append(getStructuralString());
203     if (INCLUDE_FULL_TREE_IN_TO_STRING) {
204       for (byte[] bytes : getArrays()) {
205         if (sb.length() > 0) {
206           sb.append("\n");
207         }
208         sb.append(Bytes.toString(bytes));
209       }
210     }
211     return sb.toString();
212   }
213 
214   public String getStructuralString() {
215     List<TokenizerNode> nodes = getNodes(true, true);
216     StringBuilder sb = new StringBuilder();
217     for (TokenizerNode node : nodes) {
218       String line = node.getPaddedTokenAndOccurrenceString();
219       sb.append(line + "\n");
220     }
221     return sb.toString();
222   }
223 
224 
225   /****************** get/set ************************/
226 
227   public TokenizerNode getRoot() {
228     return root;
229   }
230 
231   public int getMaxElementLength() {
232     return maxElementLength;
233   }
234 
235   public int getTreeDepth() {
236     return treeDepth;
237   }
238 
239 }