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.security.visibility;
019
020import java.nio.charset.StandardCharsets;
021import java.util.ArrayList;
022import java.util.List;
023import java.util.Stack;
024import org.apache.hadoop.hbase.security.visibility.expression.ExpressionNode;
025import org.apache.hadoop.hbase.security.visibility.expression.LeafExpressionNode;
026import org.apache.hadoop.hbase.security.visibility.expression.NonLeafExpressionNode;
027import org.apache.hadoop.hbase.security.visibility.expression.Operator;
028import org.apache.hadoop.hbase.util.Bytes;
029import org.apache.yetus.audience.InterfaceAudience;
030
031@InterfaceAudience.Private
032public class ExpressionParser {
033
034  private static final char CLOSE_PARAN = ')';
035  private static final char OPEN_PARAN = '(';
036  private static final char OR = '|';
037  private static final char AND = '&';
038  private static final char NOT = '!';
039  private static final char SPACE = ' ';
040  private static final char DOUBLE_QUOTES = '"';
041
042  public ExpressionNode parse(String expS) throws ParseException {
043    expS = expS.trim();
044    Stack<ExpressionNode> expStack = new Stack<>();
045    int index = 0;
046    byte[] exp = Bytes.toBytes(expS);
047    int endPos = exp.length;
048    while (index < endPos) {
049      byte b = exp[index];
050      switch (b) {
051        case OPEN_PARAN:
052          processOpenParan(expStack, expS, index);
053          index = skipSpaces(exp, index);
054          break;
055        case CLOSE_PARAN:
056          processCloseParan(expStack, expS, index);
057          index = skipSpaces(exp, index);
058          break;
059        case AND:
060        case OR:
061          processANDorOROp(getOperator(b), expStack, expS, index);
062          index = skipSpaces(exp, index);
063          break;
064        case NOT:
065          processNOTOp(expStack, expS, index);
066          break;
067        case DOUBLE_QUOTES:
068          int labelOffset = ++index;
069          // We have to rewrite the expression within double quotes as incase of expressions
070          // with escape characters we may have to avoid them as the original expression did
071          // not have them
072          List<Byte> list = new ArrayList<>();
073          while (index < endPos && !endDoubleQuotesFound(exp[index])) {
074            if (exp[index] == '\\') {
075              index++;
076              if (exp[index] != '\\' && exp[index] != '"') throw new ParseException(
077                "invalid escaping with quotes " + expS + " at column : " + index);
078            }
079            list.add(exp[index]);
080            index++;
081          }
082          // The expression has come to the end. still no double quotes found
083          if (index == endPos) {
084            throw new ParseException("No terminating quotes " + expS + " at column : " + index);
085          }
086          // This could be costly. but do we have any alternative?
087          // If we don't do this way then we may have to handle while checking the authorizations.
088          // Better to do it here.
089          byte[] array =
090            org.apache.hbase.thirdparty.com.google.common.primitives.Bytes.toArray(list);
091          String leafExp = Bytes.toString(array).trim();
092          if (leafExp.isEmpty()) {
093            throw new ParseException("Error parsing expression " + expS + " at column : " + index);
094          }
095          processLabelExpNode(new LeafExpressionNode(leafExp), expStack, expS, index);
096          index = skipSpaces(exp, index);
097          break;
098        default:
099          labelOffset = index;
100          do {
101            if (!VisibilityLabelsValidator.isValidAuthChar(exp[index])) {
102              throw new ParseException(
103                "Error parsing expression " + expS + " at column : " + index);
104            }
105            index++;
106          } while (index < endPos && !isEndOfLabel(exp[index]));
107          leafExp =
108            new String(exp, labelOffset, index - labelOffset, StandardCharsets.UTF_8).trim();
109          if (leafExp.isEmpty()) {
110            throw new ParseException("Error parsing expression " + expS + " at column : " + index);
111          }
112          processLabelExpNode(new LeafExpressionNode(leafExp), expStack, expS, index);
113          // We already crossed the label node index. So need to reduce 1 here.
114          index--;
115          index = skipSpaces(exp, index);
116      }
117      index++;
118    }
119    if (expStack.size() != 1) {
120      throw new ParseException("Error parsing expression " + expS);
121    }
122    ExpressionNode top = expStack.pop();
123    if (top == LeafExpressionNode.OPEN_PARAN_NODE) {
124      throw new ParseException("Error parsing expression " + expS);
125    }
126    if (top instanceof NonLeafExpressionNode) {
127      NonLeafExpressionNode nlTop = (NonLeafExpressionNode) top;
128      if (nlTop.getOperator() == Operator.NOT) {
129        if (nlTop.getChildExps().size() != 1) {
130          throw new ParseException("Error parsing expression " + expS);
131        }
132      } else if (nlTop.getChildExps().size() != 2) {
133        throw new ParseException("Error parsing expression " + expS);
134      }
135    }
136    return top;
137  }
138
139  private int skipSpaces(byte[] exp, int index) {
140    while (index < exp.length - 1 && exp[index + 1] == SPACE) {
141      index++;
142    }
143    return index;
144  }
145
146  private void processCloseParan(Stack<ExpressionNode> expStack, String expS, int index)
147    throws ParseException {
148    if (expStack.size() < 2) {
149      // When ) comes we expect atleast a ( node and another leaf/non leaf node
150      // in stack.
151      throw new ParseException();
152    } else {
153      ExpressionNode top = expStack.pop();
154      ExpressionNode secondTop = expStack.pop();
155      // The second top must be a ( node and top should not be a ). Top can be
156      // any thing else
157      if (
158        top == LeafExpressionNode.OPEN_PARAN_NODE || secondTop != LeafExpressionNode.OPEN_PARAN_NODE
159      ) {
160        throw new ParseException("Error parsing expression " + expS + " at column : " + index);
161      }
162      // a&(b|) is not valid.
163      // The top can be a ! node but with exactly child nodes. !).. is invalid
164      // Other NonLeafExpressionNode , then there should be exactly 2 child.
165      // (a&) is not valid.
166      if (top instanceof NonLeafExpressionNode) {
167        NonLeafExpressionNode nlTop = (NonLeafExpressionNode) top;
168        if (
169          (nlTop.getOperator() == Operator.NOT && nlTop.getChildExps().size() != 1)
170            || (nlTop.getOperator() != Operator.NOT && nlTop.getChildExps().size() != 2)
171        ) {
172          throw new ParseException("Error parsing expression " + expS + " at column : " + index);
173        }
174      }
175      // When (a|b)&(c|d) comes while processing the second ) there will be
176      // already (a|b)& node
177      // avail in the stack. The top will be c|d node. We need to take it out
178      // and combine as one
179      // node.
180      if (!expStack.isEmpty()) {
181        ExpressionNode thirdTop = expStack.peek();
182        if (thirdTop instanceof NonLeafExpressionNode) {
183          NonLeafExpressionNode nlThirdTop = (NonLeafExpressionNode) expStack.pop();
184          nlThirdTop.addChildExp(top);
185          if (nlThirdTop.getOperator() == Operator.NOT) {
186            // It is a NOT node. So there may be a NonLeafExpressionNode below
187            // it to which the
188            // completed NOT can be added now.
189            if (!expStack.isEmpty()) {
190              ExpressionNode fourthTop = expStack.peek();
191              if (fourthTop instanceof NonLeafExpressionNode) {
192                // Its Operator will be OR or AND
193                NonLeafExpressionNode nlFourthTop = (NonLeafExpressionNode) fourthTop;
194                assert nlFourthTop.getOperator() != Operator.NOT;
195                // Also for sure its number of children will be 1
196                assert nlFourthTop.getChildExps().size() == 1;
197                nlFourthTop.addChildExp(nlThirdTop);
198                return;// This case no need to add back the nlThirdTop.
199              }
200            }
201          }
202          top = nlThirdTop;
203        }
204      }
205      expStack.push(top);
206    }
207  }
208
209  private void processOpenParan(Stack<ExpressionNode> expStack, String expS, int index)
210    throws ParseException {
211    if (!expStack.isEmpty()) {
212      ExpressionNode top = expStack.peek();
213      // Top can not be a Label Node. a(.. is not valid. but ((a.. is fine.
214      if (top instanceof LeafExpressionNode && top != LeafExpressionNode.OPEN_PARAN_NODE) {
215        throw new ParseException("Error parsing expression " + expS + " at column : " + index);
216      } else if (top instanceof NonLeafExpressionNode) {
217        // Top is non leaf.
218        // It can be ! node but with out any child nodes. !a(.. is invalid
219        // Other NonLeafExpressionNode , then there should be exactly 1 child.
220        // a&b( is not valid.
221        // a&( is valid though. Also !( is valid
222        NonLeafExpressionNode nlTop = (NonLeafExpressionNode) top;
223        if (
224          (nlTop.getOperator() == Operator.NOT && nlTop.getChildExps().size() != 0)
225            || (nlTop.getOperator() != Operator.NOT && nlTop.getChildExps().size() != 1)
226        ) {
227          throw new ParseException("Error parsing expression " + expS + " at column : " + index);
228        }
229      }
230    }
231    expStack.push(LeafExpressionNode.OPEN_PARAN_NODE);
232  }
233
234  private void processLabelExpNode(LeafExpressionNode node, Stack<ExpressionNode> expStack,
235    String expS, int index) throws ParseException {
236    if (expStack.isEmpty()) {
237      expStack.push(node);
238    } else {
239      ExpressionNode top = expStack.peek();
240      if (top == LeafExpressionNode.OPEN_PARAN_NODE) {
241        expStack.push(node);
242      } else if (top instanceof NonLeafExpressionNode) {
243        NonLeafExpressionNode nlTop = (NonLeafExpressionNode) expStack.pop();
244        nlTop.addChildExp(node);
245        if (nlTop.getOperator() == Operator.NOT && !expStack.isEmpty()) {
246          ExpressionNode secondTop = expStack.peek();
247          if (secondTop == LeafExpressionNode.OPEN_PARAN_NODE) {
248            expStack.push(nlTop);
249          } else if (secondTop instanceof NonLeafExpressionNode) {
250            ((NonLeafExpressionNode) secondTop).addChildExp(nlTop);
251          }
252        } else {
253          expStack.push(nlTop);
254        }
255      } else {
256        throw new ParseException("Error parsing expression " + expS + " at column : " + index);
257      }
258    }
259  }
260
261  private void processANDorOROp(Operator op, Stack<ExpressionNode> expStack, String expS, int index)
262    throws ParseException {
263    if (expStack.isEmpty()) {
264      throw new ParseException("Error parsing expression " + expS + " at column : " + index);
265    }
266    ExpressionNode top = expStack.pop();
267    if (top.isSingleNode()) {
268      if (top == LeafExpressionNode.OPEN_PARAN_NODE) {
269        throw new ParseException("Error parsing expression " + expS + " at column : " + index);
270      }
271      expStack.push(new NonLeafExpressionNode(op, top));
272    } else {
273      NonLeafExpressionNode nlTop = (NonLeafExpressionNode) top;
274      if (nlTop.getChildExps().size() != 2) {
275        throw new ParseException("Error parsing expression " + expS + " at column : " + index);
276      }
277      expStack.push(new NonLeafExpressionNode(op, nlTop));
278    }
279  }
280
281  private void processNOTOp(Stack<ExpressionNode> expStack, String expS, int index)
282    throws ParseException {
283    // When ! comes, the stack can be empty or top ( or top can be some exp like
284    // a&
285    // !!.., a!, a&b!, !a! are invalid
286    if (!expStack.isEmpty()) {
287      ExpressionNode top = expStack.peek();
288      if (top.isSingleNode() && top != LeafExpressionNode.OPEN_PARAN_NODE) {
289        throw new ParseException("Error parsing expression " + expS + " at column : " + index);
290      }
291      if (!top.isSingleNode() && ((NonLeafExpressionNode) top).getChildExps().size() != 1) {
292        throw new ParseException("Error parsing expression " + expS + " at column : " + index);
293      }
294    }
295    expStack.push(new NonLeafExpressionNode(Operator.NOT));
296  }
297
298  private static boolean endDoubleQuotesFound(byte b) {
299    return (b == DOUBLE_QUOTES);
300  }
301
302  private static boolean isEndOfLabel(byte b) {
303    return (b == OPEN_PARAN || b == CLOSE_PARAN || b == OR || b == AND || b == NOT || b == SPACE);
304  }
305
306  private static Operator getOperator(byte op) {
307    switch (op) {
308      case AND:
309        return Operator.AND;
310      case OR:
311        return Operator.OR;
312      case NOT:
313        return Operator.NOT;
314      default:
315        return null;
316    }
317  }
318}